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

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2591: Spark: RewriteDatafilesAction V2

RussellSpitzer opened a new pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591


   Adds an implementation for Spark3 for performing Rewrites using the new
   action api. Only implements for Spark3 at the moment with BinPack Strategy.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()

Review comment:
       My feeling on this was that if the rewrite fails we are basically having a very serious issue since we are just trying to do a read.write operation in the underlying framework. If we want to do another attempt (and partial progress is enabled) a user can just rerun the operation and it would ignore completed binpacks. That said I don't have a strong opinion here but I would lean towards less configuration options if possible.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);

Review comment:
       I pulled out this and the group into a single function, I think that helps, but take a look in the new version




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -27,7 +27,7 @@
 import org.apache.iceberg.FileScanTask;
 import org.apache.iceberg.Table;
 
-interface RewriteStrategy extends Serializable {
+public interface RewriteStrategy extends Serializable {

Review comment:
       Moved to core




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -71,8 +71,9 @@
    * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
    * This will most likely be Action framework specific (Spark/Presto/Flink ....).
    *
+   * @param setId an identifier for this set of files

Review comment:
       Sure I was mostly copying the current usage in RewriteManager
   




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       Yeah I thought that old issue was about the coupling of ```SparkSession.close -> session.sparkContext.close``` but I don't think there is any way to explicitly cause a session to disappear without just letting it get dereferenced. My understanding was the Session object is essentially just a thin wrapper around a context with it's own runtime conf (and shared state objects it shares with other sessions), so you should be free to just make a bunch of them. I think the only ones that never get dereferenced are the SparkSession.activeSession / defaultSession linked ones.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();

Review comment:
       Can you add a comment about why this is needed? Is this to avoid a `Thread.sleep`?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > > We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at.
   > 
   > Yes I am aware of this condition, but the delete files are applied for each file scan task anyway, it's just we cannot remove it because of the condition you described, and we have to call another action, do double work to fully remove the file. Conversely, say we have another action to only remove delete files, then we are reading those delete files anyway, and it also feels wasteful to me that we have to do another bin pack after deleting those files to make the files more optimized, and potentially cause more commit conflicts.
   
   They are only applied to the files we are looking at. It's not the reading of the delete files that is expensive, to determine if a delete file isn't needed we have to scan through *every* data file they may apply to. 
   
   For example
   
   Delete file A may touch Files A, B , C ,D
   We plan for files A and B
   After running binpack we Read A and B and write A' and B' we never touch C and D.
   Delete File A may now only touch C, and D 
   
   If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   made progress.


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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       nit: extra space after `=`

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    rewriteService.shutdown();
+    committerService.shutdown();
+
+    try {
+      // All rewrites have completed and all new files have been created, we are now waiting for the commit
+      // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+      // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+      // In either case this should take much less than 10 minutes to actually complete.
+      if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+        Log.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean that " +
+            "changes were not successfully committed to the the Iceberg table.");
+      }
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    if (results.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);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroup> toGroupStream(RewriteExecutionContext ctx,
+                                          Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int globalIndex = ctx.currentGlobalIndex();
+              int partitionIndex = ctx.currentPartitionIndex(e.getKey());
+              String groupID = UUID.randomUUID().toString();
+              return new FileGroup(new FileGroupInfo(groupID, globalIndex, partitionIndex, e.getKey()), tasks);
+            }));
+  }
+
+  private void validateOptions() {
+    Set<String> validOptions = Sets.newHashSet(strategy.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 RewriteDatafiles or the strategy %s",

Review comment:
       nit: `... by the rewrite action or strategy`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    rewriteService.shutdown();
+    committerService.shutdown();
+
+    try {
+      // All rewrites have completed and all new files have been created, we are now waiting for the commit
+      // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+      // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+      // In either case this should take much less than 10 minutes to actually complete.
+      if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+        Log.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean that " +
+            "changes were not successfully committed to the the Iceberg table.");
+      }
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    if (results.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);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroup> toGroupStream(RewriteExecutionContext ctx,
+                                          Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {

Review comment:
       nit: I kept going back to the definition to find out what `e.getKey()` and `e.getValue()` means. Maybe, it will be a bit easier to read if we created separate vars with proper names.
   
   I'd also add a var for `new FileGroupInfo` as that line is quite long.
   
   ```
   return fileGroupsByPartition.entrySet().stream()
       .flatMap(e -> {
         StructLike partition = e.getKey();
         List<List<FileScanTask>> groups = e.getValue();
         return groups.stream().map(tasks -> {
           ...
           FileGroupInfo info = new FileGroupInfo(groupID, globalIndex, partitionIndex, partition);
           return new FileGroup(info, tasks);
         });
       });
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));

Review comment:
       nit: Can fit on one line?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {

Review comment:
       Do we need to catch it to immediately rethrow?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+          .mode("append")
+          .save(table.name());

Review comment:
       Do we have to locally sort data during bin-packing?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       super nit: I'd also put `ThreadFactoryBuilder` on a new line like you did in `rewriteService`.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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" +

Review comment:
       Looks like this is missing spaces at the end of files.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+

Review comment:
       nit: I'd consider the construction of `FileGroupRewriteResult` and adding it to the result map as a single logical operation. So if we are to add an empty line, I'd add it after `offer` call instead of this one.
   
   ```
   rewrittenIDs.offer(groupID);
   
   FileGroupRewriteResult fileGroupResult = ...
   results.put(groupID, ...);
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),

Review comment:
       Can we move `withJobGroupInfo` to the line above and keep `newJobGroupInfo` on this one?
   
   ```
       Set<DataFile> addedFiles = withJobGroupInfo(
           newJobGroupInfo("REWRITE-DATA-FILES", ...),
           () -> ...);
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));

Review comment:
       nit: I think this can be called `fileGroup` or `group`.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {

Review comment:
       nit: If we called `completedRewriteIDs` as `rewrittenIDs` like in `doExecute`, the condition would fit on one line.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();

Review comment:
       nit: we sometimes use `xxxID` and sometimes `xxxId` in this file.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);

Review comment:
       nit: `Ignoring a failure during ....`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);

Review comment:
       Does this mean groups that failed will not appear in the result of the action?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("Committer-Service")
+        .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    rewriteService.shutdown();
+    committerService.shutdown();
+
+    try {
+      // All rewrites have completed and all new files have been created, we are now waiting for the commit
+      // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+      // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+      // In either case this should take much less than 10 minutes to actually complete.
+      if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+        Log.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean that " +
+            "changes were not successfully committed to the the Iceberg table.");

Review comment:
       nit: double `the`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage = String.format(
+          "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that" +

Review comment:
       Missing spaces at the end of 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+

Review comment:
       Changed this around a bit, Now we only handle FileGroups within this logic




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -114,6 +117,11 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     return newDataFiles;
   }
 
+  public void clearRewrite(Table table, String fileSetID) {

Review comment:
       Couldn't this class be used for any write that we want to hijack from Spark and commit later? It may be better to name it `FileResultSetManager` or something.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(p -> p.first(), p -> p.second())));
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedSetIds fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedSetIds);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation
+   * @param setId fileSet to clean
+   */
+  protected abstract void abortFileGroup(String setId);
+
+  class Result implements RewriteDataFiles.Result {

Review comment:
       I'm not sure, but I think we could always move it when we start the other implementations. I think we'll probably be extracting some of the other functions as well




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       I like how this style made the tests readable. I've been using "check" or "assert" in method names like this. Whatever makes a readable test method is good.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitManager.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitManager(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = Sets.newHashSet();
+    Set<DataFile> addedDataFiles = Sets.newHashSet();
+    for (RewriteFileGroup group : fileGroups) {
+      rewrittenDataFiles = Sets.union(rewrittenDataFiles, group.rewrittenFiles());
+      addedDataFiles = Sets.union(addedDataFiles, group.addedFiles());
+    }
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, addedDataFiles)
+        .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(RewriteFileGroup fileGroup) {
+    Preconditions.checkState(fileGroup.addedFiles() != null,
+        "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      rewriteGroups.forEach(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private final AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          try {
+            if (completedRewrites.size() == 0) {
+              // Give other threads a chance to make progress
+              Thread.sleep(100);
+            }
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException("Interrupted while processing commits", e);
+          }
+
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.debug("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);

Review comment:
       Yeah there is a race here which is why I put the precondition check in the bottom of the commit close function. Since this is internal (ish) I didn't want to add in a synchronize block when because any calls of "offer" after "close" are an error in our implementation. The precondition is more of a warning for us writing the code that the implementation of RewriteDatafilesAction is wrong. If we change the packaging of this we could make all of these package private :(




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > > If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   > > made progress.
   > 
   > I think this is the place I am a bit confused. A' and B' don't need delete file A for sure because sequence number of A' and B' is higher. But we don't read C and D to add delete file A to C and D's FileScanTask. It's done by reading the statistics of delete file A and determined by the partition filter. As long as there are files of lower sequence number in that partition, the delete file will be included to that file scan task.
   > 
   > This means that if we can have a counter for each delete file and expose a method `cleanUnreferencedDeleteFiles()` called after `planFileGroups()`, we can naturally get all the files compacted just by running bin packing continuously.
   
   We don't read C and D to check if delete file A is used in C and D's scan task. But we don't know if Delete File A is needed without reading C and D. Delete File A may not touch any files in C and D or it may. We can count references, but this ends up being a problem if we don't actually include all files because we have pushed down our filters size filters to the manifest reader itself (future plans) and it only helps us in the luckiest case where we know a delete file has been completely dereferenced. If file C for example is the correct size, and we never need to rewrite it, we never clean up those deletes. Meaning we still have to make another sort of action to clean up those files.
   
   That said, the same sweep of delete-files you are describing should probably just be performed on every action which deletes files, and not just during bin pack since that particular method of eliminating files is very cheap but also unlikely to actually pick up any delete files for removal. We talked about this previously as a possible post-merge, post-delete, post-rewrite sort of thing. But again since that kind of clean up never conflicts, we can really do it at any time without a lot of cost to perf.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   @aokolnychyi I'm in favor of merging early and often :) 


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       The Spark Session Objects? They should just get garbaged collected as soon as this function is done. Unless we are checking something else? It should be using sharedstate for everything that had a explicitly managed lifecycle




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       I'm not sure what you are asking here, this method's doc
   ```Create an identical copy of this SparkSession, sharing the underlying SparkContext and shared state. All the state of this session (i.e. SQL configurations, temporary tables, registered functions) is copied over, and the cloned session is set up with the same shared state as this session. The cloned session is independent of this session, that is, any non-global change in either session is not reflected in the other.```
   
   We basically just use this make sure we don't change the user's spark session config




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       This is safe? I thought sessions couldn't be closed?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();

Review comment:
       Just wanted to not have this thread spinning up a core to 100% if we could get other threads in, I can add in a sleep to make it more clear.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +

Review comment:
       I need to add that to the "commit" portion. Here I'm not sure it's the right decision. This is where we actually had an error writing (or reading) and not it committing.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()

Review comment:
       sounds good to me, going to use the name RewriteExecutionContext just to emphasize ... maybe we can drop rewrite though ... execution context just felt to generic to me




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()

Review comment:
       There is two situations we have to be careful with.
   
   First, the result file size may be a little bit bigger than our target file size. We don't want to cut another file with just a couple of MBs. It is better to write a slightly bigger file, especially because we have the max file size threshold.
   
   Second, I have seen a number of use cases when the file size after compaction is substantially larger than what we anticipated. I suspect it is related to not being able to apply a specific encoding technique as the size of a row group grows. There were use cases when there was more than 100% mismatch. 
   
   I think these two use cases can be solved by limiting our file size in the compaction as let's say 0.8 (or any other value) of the max file size. This can be done by passing a dedicated write option.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()

Review comment:
       There are two situations we have to be careful with.
   
   First, the result file size may be a little bit bigger than our target file size. We don't want to cut another file with just a couple of MBs. It is better to write a slightly bigger file, especially because we have the max file size threshold.
   
   Second, I have seen a number of use cases when the file size after compaction is substantially larger than what we anticipated. I suspect it is related to not being able to apply a specific encoding technique as the size of a row group grows. There were use cases when there was more than 100% mismatch. 
   
   I think these two use cases can be solved by limiting our file size in the compaction as let's say 0.8 (or any other value) of the max file size. This can be done by passing a dedicated write option.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);

Review comment:
       Too bad checkstyle can't check the logging string formats too :(




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {

Review comment:
       It's annoying because "rewriteFiles" will be strategy specific and framework specific but all the other methods should just be framework specific




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Map;
+
+public class BaseRewriteDataFilesResult implements RewriteDataFiles.Result {
+  private final Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap;
+
+  public BaseRewriteDataFilesResult(
+      Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap) {

Review comment:
       What if we import `FileGroupInfo` and `FileGroupRewriteResult` directly? Then this can fit on one line.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));

Review comment:
       Does this need to use `Tasks`? `suppressFailureWhenFinished` is already used in `abortFileGroup` so I don't think that both methods need to use `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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Map;
+
+public class BaseRewriteDataFilesResult implements RewriteDataFiles.Result {
+  private final Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap;
+
+  public BaseRewriteDataFilesResult(
+      Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap) {

Review comment:
       Should we remove our checkstyle rules on static imports? I feel like we are constantly adding exclusions




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();

Review comment:
       Maybe, we should replace the second stream API part?
   
   ```
     private Map<StructLike, List<List<FileScanTask>>> planFileGroups(RewriteStrategy strategy) {
       CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
           .filter(filter)
           .ignoreResiduals()
           .planFiles();
   
       try {
         Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
             .collect(Collectors.groupingBy(task -> task.file().partition()));
   
         Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
   
         filesByPartition.forEach((partition, tasks) -> {
           List<List<FileScanTask>> fileGroups = toFileGroups(tasks, strategy);
           if (fileGroups.size() > 0) {
             fileGroupsByPartition.put(partition, fileGroups);
           }
         });
   
         return fileGroupsByPartition;
       } finally {
         try {
           fileScanTasks.close();
         } catch (IOException io) {
           LOG.error("Cannot properly close file iterable while planning for rewrite", io);
         }
       }
     }
   ```




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);

Review comment:
       Yep sorry didn't push that fix yet, I noticed this on my last run through




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);
+    }
+
+    /**
+     * Returns all File groups which have been committed
+     */
+    public List<RewriteFileGroup> results() {
+      Preconditions.checkState(committerService.isShutdown(),
+          "Cannot get results from a service which has not been closed");
+      return committedRewrites;
+    }
+
+    @Override
+    public void close() {
+      Preconditions.checkState(running.compareAndSet(true, false),
+          "Cannot close already closed RewriteService");
+      LOG.info("Closing commit service for {}", table);
+      committerService.shutdown();
+
+      try {
+        // All rewrites have completed and all new files have been created, we are now waiting for the commit
+        // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+        // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+        // In either case this should take much less than 10 minutes to actually complete.
+        if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+          LOG.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean " +
+              "that changes were not successfully committed to the the Iceberg table.");
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);

Review comment:
       Yeah, here's some info: https://www.baeldung.com/java-interrupted-exception#2-restore-the-interrupt




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);

Review comment:
       yep, That was from me debugging I forgot to lower the severity when i was done




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));

Review comment:
       Shouldn't the result be propagated during the commit? Also, what if a group fails? What should be in the result map?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       At least on Parquet, differences in compression and encoding seem to be issues here. @aokolnychyi has more info but one of the hypothesis was that smaller files used dictionary encoding while larger files did not.
   
   Most of the experience with this is from production use-cases with users with large numbers of small 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.

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



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


[GitHub] [iceberg] jackye1995 commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   made progress.
   
   I think this is the place I am a bit confused. A' and B' don't need delete file A for sure because sequence number of A' and B' is higher. But we don't read C and D to add delete file A to C and D's FileScanTask. It's done by reading the statistics of delete file A and determined by the partition filter. As long as there are files of lower sequence number in that partition, the delete file will be included to that file scan task.
   
   This means that if we can have a counter for each delete file and expose a method `cleanUnreferencedDeleteFiles()` called after `planFileGroups()`, we can naturally get all the files compacted just by running bin packing continuously.


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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Non-Spark Changes - https://github.com/apache/iceberg/pull/2770


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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       Rewrote the documentation here to hopefully make it more clear




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);

Review comment:
       nit: 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] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+

Review comment:
       nit: extra line?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       There is no other way to close a session so i'm not sure what else we can do, since session.close just closes the sparkcontext.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/test/java/org/apache/iceberg/actions/TestBinPackStrategy.java
##########
@@ -60,7 +60,7 @@ public Table table() {
     }
 
     @Override
-    public Set<DataFile> rewriteFiles(String groupId, List<FileScanTask> filesToRewrite) {

Review comment:
       We are pulling this because we removed "groupID" state and put it into the strategy implementations




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());

Review comment:
       nit: I think having `collect` on a new line would be cleaner.

##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());

Review comment:
       nit: I think having `collect` on a new line would be cleaner.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSpark3Action.java
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BaseRewriteDataFilesSpark3Action extends BaseRewriteDataFilesSparkAction {
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSpark3Action.class);
+  private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get();
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  protected BaseRewriteDataFilesSpark3Action(SparkSession spark, Table table) {
+    super(spark, table);
+  }
+
+  @Override
+  protected RewriteStrategy defaultStrategy() {
+    return new Spark3BinPackStrategy(table(), spark());

Review comment:
       Yeah this is a bit behind some of our other discussions on this, In my local copy this no longer exists. @rdblue , @aokolnychyi and I discussed passing strategies, especially those with more complicated args which is why RewriteDataFiles now has binPack() as a method.
   
   In my local version with binPack(), sort() and sort(SortOrder sortOrder) call on methods here of either
   ```java
   binPackStrategy()
   //or
   sortStrategy() 
   ```
   And then modify the resultant strategy with their options.




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

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



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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       Does this happen on all file formats? What file format did you use for this test?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );

Review comment:
       nit: shall we move `);` to the line above?




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

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



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


[GitHub] [iceberg] jackye1995 edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   made progress.
   
   I think this is the place I am a bit confused. A' and B' don't need delete file A for sure because sequence number of A' and B' is higher. But we don't read C and D to add delete file A to C and D's FileScanTask. It's done by reading the statistics of delete file A and determined by the partition filter. As long as there are files of lower sequence number in that partition, the delete file will be included to that file scan task if data filter also pass.
   
   This means that if we can have a counter for each delete file and expose a method `cleanUnreferencedDeleteFiles()` called after `planFileGroups()`, we can naturally get all the files compacted just by running bin packing continuously.


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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();

Review comment:
       Is this needed?




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Thanks, @RussellSpitzer! I merged this. And thanks to everyone that helped review!


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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -42,7 +42,7 @@
  * more files than {@link MIN_INPUT_FILES} or would produce at least one file of

Review comment:
       Thanks, I didn't know 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.

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



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


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       Thanks for the information! I was wondering whether this could happen to Avro as well since it is a row-based file format and it might not have the dictionary encoding.




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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   @jackye1995 @rdblue @stevenzwu @chenjunjiedada Here is the big one, Implementation 95% there, tests are in progress


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean

Review comment:
       Nit: `fileSet` looks like a variable name. Is there a better way to refer to 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();

Review comment:
       changed to completedRewriteIds




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       I'm really used to Scalatest where everything is "X should Y" so I tend to name my assertion things "should". This function is actually for the Sort Rewrite Action i'll be adding next to OSS. Basically makes sure that our "Sort" actually sorted.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   I agree about the complexity of figuring out whether a delete file is still needed and that it is not really specific to compaction. Ideally, the algorithm should be generic and efficient enough so that we can apply it beyond compaction use cases. I think having partition-level summaries may help us there (i.e. knowing the min sequence of data files per partition).
   
   There is probably another question that we can consider sooner (but after this PR). We could pick a file if it is optimal in size but requires us to apply a lot of delete files on scan. It probably makes sense to include such data files for rewrites as the new file will have a higher sequence number so the deletes will no longer apply. Again, that's something we can add in the future.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {

Review comment:
       Sure? I don't think it matters as long as it follows the parent.




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+

Review comment:
       Nit: extra newline.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.
+   * This helps us in the case where our estimate for the task size is under the target size but the
+   * actual written file size is slightly larger.
+   * @return the target size plus one half of the distance between max and target
+   */
+  protected long writeMaxFileSize() {
+    return (long) (this.targetFileSize + ((this.maxFileSize - this.targetFileSize) * 0.5));

Review comment:
       I am afraid we cannot do much in that case




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



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

Review comment:
       Do we need a separate var for this and then keep it in sync with `outputFiles`?

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

Review comment:
       nit: same here




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())

Review comment:
       Nit: missing newline after control flow.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);
+    }
+
+    /**
+     * Returns all File groups which have been committed
+     */
+    public List<RewriteFileGroup> results() {
+      Preconditions.checkState(committerService.isShutdown(),
+          "Cannot get results from a service which has not been closed");
+      return committedRewrites;
+    }
+
+    @Override
+    public void close() {
+      Preconditions.checkState(running.compareAndSet(true, false),
+          "Cannot close already closed RewriteService");
+      LOG.info("Closing commit service for {}", table);
+      committerService.shutdown();
+
+      try {
+        // All rewrites have completed and all new files have been created, we are now waiting for the commit
+        // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+        // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+        // In either case this should take much less than 10 minutes to actually complete.
+        if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+          LOG.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean " +
+              "that changes were not successfully committed to the the Iceberg table.");
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);

Review comment:
       I think this should also set the thread's interrupt flag to signal that it was interrupted.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+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.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo;
+import org.apache.iceberg.actions.BaseRewriteDataFilesResult;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.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.collect.Streams;
+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.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<RewriteFileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new BaseRewriteDataFilesResult(Collections.emptyList());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DataFile> addedFiles = withJobGroupInfo(
+        newJobGroupInfo("REWRITE-DATA-FILES", desc),
+        () -> strategy.rewriteFiles(fileGroup.fileScans()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Files Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  @VisibleForTesting
+  RewriteDataFilesCommitManager commitUtil() {

Review comment:
       Shall this be called `commitManager` after the recent rename?

##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupRewriteResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewriteDataFiles.FileGroupRewriteResult;
+
+public class BaseRewriteDataFilesFileGroupRewriteResult implements FileGroupRewriteResult {

Review comment:
       nit: just `BaseFileGroupRewriteResult`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+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.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo;
+import org.apache.iceberg.actions.BaseRewriteDataFilesResult;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFilesCommitManager;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.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.collect.Streams;
+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.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<RewriteFileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new BaseRewriteDataFilesResult(Collections.emptyList());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DataFile> addedFiles = withJobGroupInfo(
+        newJobGroupInfo("REWRITE-DATA-FILES", desc),
+        () -> strategy.rewriteFiles(fileGroup.fileScans()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Files Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  @VisibleForTesting
+  RewriteDataFilesCommitManager commitUtil() {

Review comment:
       Variables that refer to it should be renamed too.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       I did look into the code a little bit. Seems the current assumption is that the state will be GCed once a session becomes unreachable?

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

Review comment:
       It looks like we use this class in debug messages but don't provide a reasonable `toString` implementation.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {

Review comment:
       This should happen if the maxGroup size is not equally divisible, whether or not partial progress is enabled. If PartialProgress is enabled it will just be the remainder of the last group. If it isn't enabled this will be the entire set of all groups.




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

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



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


[GitHub] [iceberg] jccsjtu commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.
+   * This helps us in the case where our estimate for the task size is under the target size but the
+   * actual written file size is slightly larger.
+   * @return the target size plus one half of the distance between max and target
+   */
+  protected long writeMaxFileSize() {
+    return (long) (this.targetFileSize + ((this.maxFileSize - this.targetFileSize) * 0.5));

Review comment:
       if user set the this.maxFileSize = this.targetFileSize, may still be spilt




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.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 org.apache.iceberg.StructLike;
+
+public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo {

Review comment:
       Not sure what do o here for equals and hashcode, since we can't get those for a StructLike without a Struct and at this point we don't have them. I'd be fine with changing the interface to just be a list of Pairs instead of key/value (or some new result type which has file group embedded)
   
   Like
   ```
   FileGroupRewriteResult {
     info
     filesAdded
     filesRewritten
     }
     ```




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(

Review comment:
       just trying to sneak my preferred indentation in :)




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+
+    fileSetIDs.stream().map(id -> toID(table, id)).forEach(resultMap::remove);

Review comment:
       As noted above I think we'll do a separate PR to clean up this file post merge.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+
+    fileSetIDs.stream().map(id -> toID(table, id)).forEach(resultMap::remove);

Review comment:
       As noted above I think we'll do a separate PR to clean up this file (and remove unneeded tests) post merge.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupRewriteResult.java
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+public class BaseRewriteDataFilesFileGroupRewriteResult implements RewriteDataFiles.FileGroupRewriteResult {
+  private final int addedDataFilesCount;
+  private final int rewrittenDataFilesCount;
+
+  public BaseRewriteDataFilesFileGroupRewriteResult(int addedDataFilesCount, int rewrittenDataFilesCount) {
+    this.addedDataFilesCount = addedDataFilesCount;
+    this.rewrittenDataFilesCount = rewrittenDataFilesCount;
+  }
+
+  @Override
+  public int addedDataFilesCount() {
+    return this.addedDataFilesCount;

Review comment:
       nit: redundant `this`?

##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupRewriteResult.java
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+public class BaseRewriteDataFilesFileGroupRewriteResult implements RewriteDataFiles.FileGroupRewriteResult {
+  private final int addedDataFilesCount;
+  private final int rewrittenDataFilesCount;
+
+  public BaseRewriteDataFilesFileGroupRewriteResult(int addedDataFilesCount, int rewrittenDataFilesCount) {
+    this.addedDataFilesCount = addedDataFilesCount;
+    this.rewrittenDataFilesCount = rewrittenDataFilesCount;
+  }
+
+  @Override
+  public int addedDataFilesCount() {
+    return this.addedDataFilesCount;
+  }
+
+  @Override
+  public int rewrittenDataFilesCount() {
+    return this.rewrittenDataFilesCount;

Review comment:
       nit: redundant `this`?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.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 org.apache.iceberg.StructLike;
+
+public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo {

Review comment:
       Do we need to override `equals` and `hashCode` as this is used in the result map?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSpark3.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.actions.rewrite.Spark3BinPackStrategy;
+import org.apache.spark.sql.SparkSession;
+
+public class RewriteDataFilesSpark3 extends BaseRewriteDataFilesSparkAction {

Review comment:
       I can change this but is it a Base class if it's the leaf implementation? I know we have that on the other extensions so I'll do it here too.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



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

Review comment:
       Yep no longer needed, 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();

Review comment:
       Not yet, I was going put this in the job description as well, ie Partition X : # Y of Total
   




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -27,7 +27,7 @@
 import org.apache.iceberg.FileScanTask;
 import org.apache.iceberg.Table;
 
-interface RewriteStrategy extends Serializable {
+public interface RewriteStrategy extends Serializable {

Review comment:
       Shall we move it to core for now? I think it is a bit too early to expose it. 

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);

Review comment:
       Would it make sense to put this into a separate method to simplify the collector statement?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(

Review comment:
       Typo? `committerService`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {

Review comment:
       Or we actually assume this only happens if all rewrites have completed?

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -71,8 +71,9 @@
    * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
    * This will most likely be Action framework specific (Spark/Presto/Flink ....).
    *
+   * @param setId an identifier for this set of files
    * @param filesToRewrite a group of files to be rewritten together
    * @return a list of newly written files
    */
-  List<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite);
+  Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite);

Review comment:
       nit: `setID` 

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());

Review comment:
       nit: extra space, do we need an explicit `this` in `this.options()`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()

Review comment:
       Seems like we don't close this `Iterable` anywhere?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =

Review comment:
       Can we split this method into smaller ones? I think planning file groups could definitely be in another method.

##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableIterable.java
##########
@@ -26,9 +26,11 @@
 import java.util.NoSuchElementException;
 import java.util.function.Function;
 import java.util.function.Predicate;
+import java.util.stream.Stream;

Review comment:
       Not needed?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {

Review comment:
       I'd consider putting this into a separate method and calling it from map.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));

Review comment:
       nit: I think it is a bit more readable to have `.stream()` on the same line with `=` and then each subsequent action on a separate line.
   
   ```
   ... filesByPartition = Streams.stream(files)
        .collect(...);
   ```
   

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();

Review comment:
       nit: `Maps.newHashMap`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();

Review comment:
       Is this used anywhere?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(

Review comment:
       Here as well. It may be a bit easier to read if Collectors.toMap was on a single separate line.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();

Review comment:
       Is there a better prefix we can give? `my` -> `current` or something?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);

Review comment:
       nit: shall we define temp vars for UUID.randomUUID() and FileGroupInfo?
   
   ```
   String xxx = UUID.randomUUID().toString();
   FileGroupInfo yyy = new FileGroupInfo(xxx, myJobIndex, myPartIndex, e.getKey());
   return Pair.of(yyy, tasks);
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();

Review comment:
       `completedRewrite` -> `completedRewrites`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);

Review comment:
       Can we move `totalGroups` closer to the place it is used?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {

Review comment:
       Should this happen only if partial progress enabled?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;

Review comment:
       nit: make final and move above mutable vars?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {

Review comment:
       Shall we stop other rewrites if this happens?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {

Review comment:
       I am not sure I understand the second branch of this condition. Suppose I have 10 partitions and allow only 2 commits at most. If I managed to compact 9 partitions and one failed, we should still commit the remaining 4?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSpark3.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.actions.rewrite.Spark3BinPackStrategy;
+import org.apache.spark.sql.SparkSession;
+
+public class RewriteDataFilesSpark3 extends BaseRewriteDataFilesSparkAction {
+  private FileRewriteCoordinator coordinator = FileRewriteCoordinator.get();

Review comment:
       nit: final?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(p -> p.first(), p -> p.second())));
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedSetIds fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedSetIds);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation
+   * @param setId fileSet to clean
+   */
+  protected abstract void abortFileGroup(String setId);
+
+  class Result implements RewriteDataFiles.Result {

Review comment:
       Would it make sense to move this to core where we have other results?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {

Review comment:
       Can we put the commit closure into a separate method?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSpark3.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.actions.rewrite.Spark3BinPackStrategy;
+import org.apache.spark.sql.SparkSession;
+
+public class RewriteDataFilesSpark3 extends BaseRewriteDataFilesSparkAction {

Review comment:
       `BaseRewriteDataFilesSpark3Action`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);

Review comment:
       Can we group all abstract method together? I think they should also be below the constructor. 

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, setId, filesToRewrite);
+
+    // read and pack original 4 files into 2 splits
+    Dataset<Row> scanDF = spark.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, setId)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    try {
+      scanDF.write()

Review comment:
       I think we should clone the original session and disable AQE to avoid any surprises. 

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {

Review comment:
       I think we should annotate each job with proper Spark UI.
   
   ```
   Rewriting files (bin-pack, partition '$partition', jobIndex/jobs, partitionIndex/partitions) in db.tbl
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(p -> p.first(), p -> p.second())));

Review comment:
       nit: `Collectors.toMap(Pair::first, Pair::second)`

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {

Review comment:
       Shouldn't it be `>=`, though? We should be able to commit if we have enough groups, no?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);

Review comment:
       A helper error message?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, setId, filesToRewrite);
+
+    // read and pack original 4 files into 2 splits

Review comment:
       nit: wrong comment?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(p -> p.first(), p -> p.second())));
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedSetIds fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedSetIds);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation
+   * @param setId fileSet to clean
+   */
+  protected abstract void abortFileGroup(String setId);

Review comment:
       Let's move this directly after the constructor together with other abstract methods.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {

Review comment:
       Wait, I think that's handled below.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroupForRewrite> toGroupStream(RewriteExecutionContext ctx,
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int myJobIndex = ctx.currentGlobalIndex();

Review comment:
       switched to those - current so the "new FileGroup" fits on one line




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATAFILES", desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      int totalGroups, Map<StructLike, Integer> groupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair -> rewriteFiles(infoListPair, totalGroups, groupsPerPartition, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      int totalGroups, Map<StructLike, Integer> groupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID); })
+        .run(infoListPair -> rewriteFiles(infoListPair, totalGroups, groupsPerPartition, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<Pair<FileGroupInfo, List<FileScanTask>>> toJobStream(
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int myJobIndex = jobIndex.getAndIncrement();
+              int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+              String groupID = UUID.randomUUID().toString();
+              return Pair.of(new FileGroupInfo(groupID, myJobIndex, myPartIndex, e.getKey()), tasks);
+            }));
+  }
+
+  private void validateOptions() {
+
+    Set<String> validOptions = Sets.newHashSet(strategy.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 RewriteDatafiles or the strategy %s",
+        invalidKeys, strategy.name());
+
+    maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS_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);
+
+    Preconditions.checkArgument(maxConcurrentFileGroupActions >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS, maxConcurrentFileGroupActions);
+
+    Preconditions.checkArgument(!partialProgressEnabled || 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(FileGroupInfo fileGroupInfo, int totalGroups,
+      int numFilesToRewrite, int numFilesPerPartition, String strategyName) {

Review comment:
       changed this to also use new container classes, so none of these args exist now




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+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.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo;
+import org.apache.iceberg.actions.BaseRewriteDataFilesResult;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.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.collect.Streams;
+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.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<RewriteFileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new BaseRewriteDataFilesResult(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DataFile> addedFiles = withJobGroupInfo(
+        newJobGroupInfo("REWRITE-DATA-FILES", desc),
+        () -> strategy.rewriteFiles(fileGroup.fileScans()));
+
+    fileGroup.outputFiles(addedFiles);
+    LOG.info("Rewrite Files Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  @VisibleForTesting
+  RewriteDataFilesCommitUtil commitUtil() {
+    return new RewriteDataFilesCommitUtil(table);
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) {
+    ExecutorService rewriteService = rewriteService();
+    RewriteDataFilesCommitUtil commitUtil = commitUtil();
+
+    ConcurrentLinkedQueue<RewriteFileGroup> rewrittenGroups = Queues.newConcurrentLinkedQueue();
+    ConcurrentMap<FileGroupInfo, FileGroupRewriteResult> results = Maps.newConcurrentMap();
+
+    Tasks.Builder<RewriteFileGroup> 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(rewriteFiles(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(group -> commitUtil.abortFileGroup(group));
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitUtil.commitOrClean(Sets.newHashSet(rewrittenGroups));
+      rewrittenGroups.forEach(group -> results.put(group.info(), group.asResult()));
+    } 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);
+    }
+
+    return new BaseRewriteDataFilesResult(Maps.newHashMap(results));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewriteDataFilesCommitUtil.CommitService commitService = commitUtil().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(rewriteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewriteFileGroup> 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);
+    }
+
+    return new BaseRewriteDataFilesResult(commitResults.stream()
+        .collect(Collectors.toMap(RewriteFileGroup::info, RewriteFileGroup::asResult)));
+  }
+
+  private Stream<RewriteFileGroup> toGroupStream(RewriteExecutionContext ctx,
+                                                 Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> {

Review comment:
       nit: move `e -> {` to the line above and then reformat the block below? Will reduce the indentation in the closure. 




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()

Review comment:
       nit: formatting




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, setId, filesToRewrite);
+
+    // read and pack original 4 files into 2 splits
+    Dataset<Row> scanDF = spark.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, setId)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, setId)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    return rewriteCoordinator.fetchNewDataFiles(table, ImmutableSet.of(setId));

Review comment:
       I think we don't properly clean up references in `FileScanTaskSetManager` and `FileRewriteCoordinator`. We could invalidate `FileRewriteCoordinator` in a finally block just above and we could make `FileRewriteCoordinator$commit` return a set of new 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {

Review comment:
       I think we can ignore this for now and wait for more feedback.




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,

Review comment:
       nit: `xxxRewrite` -> `xxxRewrites` or `xxxRewriteIDs` or `rewrittenGroupIDs`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {

Review comment:
       Should it return the interface?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();

Review comment:
       nit: looks like this name does not match what we have in `doExecuteWithPartialProgress`.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,

Review comment:
       nit: Shall this be placed into a separate method? Can we also format it similar to what we do in `ThreadPools`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroupForRewrite> toGroupStream(RewriteExecutionContext ctx,
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int myJobIndex = ctx.currentGlobalIndex();
+              int myPartIndex = ctx.currentPartitionIndex(e.getKey());
+              String groupID = UUID.randomUUID().toString();
+              return new FileGroupForRewrite(new FileGroupInfo(groupID, myJobIndex, myPartIndex, e.getKey()), tasks);
+            }));
+  }
+
+  private void validateOptions() {
+    Set<String> validOptions = Sets.newHashSet(strategy.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 RewriteDatafiles or the strategy %s",
+        invalidKeys, strategy.name());
+
+    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);
+
+    Preconditions.checkArgument(maxConcurrentFileGroupRewrites >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_REWRITES, maxConcurrentFileGroupRewrites);
+
+    Preconditions.checkArgument(!partialProgressEnabled || 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(FileGroupForRewrite group, RewriteExecutionContext ctx) {
+    StructLike partition = group.partition();
+
+    return String.format("Rewrite %s, FileGroup %d/%d : Partition %s:%d/%d : Rewriting %d files : Table %s",
+        strategy.name(), group.globalIndex(), ctx.totalGroupCount(), partition, group.partitionIndex(),
+        ctx.groupsInPartition(partition), group.numFiles(), table.name());
+  }
+
+  class Result implements RewriteDataFiles.Result {
+    private final Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap;
+
+    Result(
+        Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap) {
+      this.resultMap = resultMap;
+    }
+
+    @Override
+    public Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap() {
+      return resultMap;
+    }
+  }
+
+  class FileGroupInfo implements RewriteDataFiles.FileGroupInfo {
+
+    private final String groupID;
+    private final int globalIndex;
+    private final int partitionIndex;
+    private final StructLike partition;
+
+    FileGroupInfo(String groupID, int globalIndex, int partitionIndex, StructLike partition) {
+      this.groupID = groupID;
+      this.globalIndex = globalIndex;
+      this.partitionIndex = partitionIndex;
+      this.partition = partition;
+    }
+
+    @Override
+    public int globalIndex() {
+      return globalIndex;
+    }
+
+    @Override
+    public int partitionIndex() {
+      return partitionIndex;
+    }
+
+    @Override
+    public StructLike partition() {
+      return partition;
+    }
+
+    @Override
+    public String toString() {
+      return "FileGroupInfo{" +
+          "groupID=" + groupID +
+          ", globalIndex=" + globalIndex +
+          ", partitionIndex=" + partitionIndex +
+          ", partition=" + partition +
+          '}';
+    }
+
+    public String groupID() {
+      return groupID;
+    }
+  }
+
+  @VisibleForTesting
+  class FileGroupRewriteResult implements RewriteDataFiles.FileGroupRewriteResult {
+    private final int addedDataFilesCount;
+    private final int rewrittenDataFilesCount;
+
+    FileGroupRewriteResult(int addedDataFilesCount, int rewrittenDataFilesCount) {
+      this.addedDataFilesCount = addedDataFilesCount;
+      this.rewrittenDataFilesCount = rewrittenDataFilesCount;
+    }
+
+    @Override
+    public int addedDataFilesCount() {
+      return this.addedDataFilesCount;
+    }
+
+    @Override
+    public int rewrittenDataFilesCount() {
+      return this.rewrittenDataFilesCount;
+    }
+  }
+
+  static class FileGroupForRewrite {

Review comment:
       nit: just `FileGroup`? Should be shorter and still descriptive enough.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =

Review comment:
       nit: What about putting vars on different lines?
   
   ```
       Set<DataFile> addedFiles = withJobGroupInfo(
           newJobGroupInfo("REWRITE-DATA-FILES", desc),
           () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())

Review comment:
       Shall we overload `foreach` to also accept `Stream`? We support a number of things there.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();

Review comment:
       nit: Shall we move this up into a var like we have in `BaseRewriteDataFilesSpark3Action`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,

Review comment:
       If we had a method for constructing a rewrite service, we could use it here too.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));

Review comment:
       nit: I think it would fit on one line if it was called `fileGroup`.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {

Review comment:
       Who is responsible for cleaning up resources? It seems we do that in two places: in strategy and in the action. I think we will have to eventually either delegate more to the strategy interface or create a rewriter interface that would be responsible for rewriting, committing, aborting.
   
   Right now both Spark 3 action as well as as Spark 3 bin packing strategy interact with the task set manger and commit coordinator. Ideally, I'd have one just one entity doing that.

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {
+        rewriteCoordinator.abortRewrite(table, groupID);
+        manager.removeTasks(table, groupID);
+      } finally {
+        throw new RuntimeException("Cannot complete rewrite, an exception was thrown during the write operation", e);

Review comment:
       Do we have to wrap this into a runtime exception or is it just enough to rethrow? We add an error message in doExecute.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));

Review comment:
       nit: `fileGroup`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(

Review comment:
       nit: What about formatting like [here](https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java#L94)?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroupForRewrite> toGroupStream(RewriteExecutionContext ctx,
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int myJobIndex = ctx.currentGlobalIndex();

Review comment:
       nit: `currentGlobalIndex`, `currentPartitionIndex`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +

Review comment:
       Shall we advise folks to consider enabling partial progress like we discussed?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })

Review comment:
       nit: `})` on another line?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<FileGroupForRewrite> toGroupStream(RewriteExecutionContext ctx,
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {

Review comment:
       nit: arg formatting

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java
##########
@@ -58,9 +60,19 @@ public void stageTasks(Table table, String setID, List<FileScanTask> tasks) {
     return tasksMap.remove(id);
   }
 
-  private Pair<String, String> toID(Table table, String setID) {
+  public Set<String> fetchSets(Table table) {

Review comment:
       nit: `fetchSetIDs`?

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java
##########
@@ -58,9 +60,19 @@ public void stageTasks(Table table, String setID, List<FileScanTask> tasks) {
     return tasksMap.remove(id);
   }
 
-  private Pair<String, String> toID(Table table, String setID) {
+  public Set<String> fetchSets(Table table) {
+    return tasksMap.keySet().stream()
+        .filter(e -> e.first().equals(toID(table)))
+        .map(Pair::second)
+        .collect(Collectors.toSet());
+  }
+
+  private String toID(Table table) {

Review comment:
       nit: tableUUID(Table 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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   @jackye1995 @rdblue @stevenzwu @chenjunjiedada Here is the big one, Implementation, tests are in progress


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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, setId, filesToRewrite);
+
+    // read and pack original 4 files into 2 splits
+    Dataset<Row> scanDF = spark.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, setId)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, setId)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    return rewriteCoordinator.fetchNewDataFiles(table, ImmutableSet.of(setId));

Review comment:
       I think we don't properly clean up references in `FileScanTaskSetManager` and `FileRewriteCoordinator`. We could invalidate `FileRewriteCoordinator` in a finally block just above and we could make `commit` in `FileRewriteCoordinator` return a set of new 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+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.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo;
+import org.apache.iceberg.actions.BaseRewriteDataFilesResult;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.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.collect.Streams;
+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.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<RewriteFileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new BaseRewriteDataFilesResult(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DataFile> addedFiles = withJobGroupInfo(
+        newJobGroupInfo("REWRITE-DATA-FILES", desc),
+        () -> strategy.rewriteFiles(fileGroup.fileScans()));
+
+    fileGroup.outputFiles(addedFiles);
+    LOG.info("Rewrite Files Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  @VisibleForTesting
+  RewriteDataFilesCommitUtil commitUtil() {
+    return new RewriteDataFilesCommitUtil(table);
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) {
+    ExecutorService rewriteService = rewriteService();
+    RewriteDataFilesCommitUtil commitUtil = commitUtil();
+
+    ConcurrentLinkedQueue<RewriteFileGroup> rewrittenGroups = Queues.newConcurrentLinkedQueue();
+    ConcurrentMap<FileGroupInfo, FileGroupRewriteResult> results = Maps.newConcurrentMap();
+
+    Tasks.Builder<RewriteFileGroup> 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(rewriteFiles(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(group -> commitUtil.abortFileGroup(group));
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitUtil.commitOrClean(Sets.newHashSet(rewrittenGroups));
+      rewrittenGroups.forEach(group -> results.put(group.info(), group.asResult()));
+    } 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);
+    }
+
+    return new BaseRewriteDataFilesResult(Maps.newHashMap(results));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewriteDataFilesCommitUtil.CommitService commitService = commitUtil().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(rewriteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewriteFileGroup> 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);
+    }
+
+    return new BaseRewriteDataFilesResult(commitResults.stream()
+        .collect(Collectors.toMap(RewriteFileGroup::info, RewriteFileGroup::asResult)));
+  }
+
+  private Stream<RewriteFileGroup> toGroupStream(RewriteExecutionContext ctx,
+                                                 Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+
+    // Todo Add intelligence to the order in which we do rewrites instead of just using partition order
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> {
+              StructLike partition = e.getKey();
+              List<List<FileScanTask>> fileGroups = e.getValue();
+              return fileGroups.stream().map(tasks -> {
+                int globalIndex = ctx.currentGlobalIndex();
+                int partitionIndex = ctx.currentPartitionIndex(partition);
+                FileGroupInfo info = new BaseRewriteDataFilesFileGroupInfo(globalIndex, partitionIndex, partition);
+                return new RewriteFileGroup(info, tasks);
+              });
+            });
+  }
+
+  private void validateOptions() {

Review comment:
       nit: `validateAndInitOptions` as it not only validates but also inits them?




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

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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       I should clarify, more my confusion was less with `should` and more that it was `shouldHaveLastCommitSorted`, but the assertion is `Found overlapping files`.
   
   I guess I would have expected `shouldHaveNoOverlappingFiles`. or something.
   
   I agree tha the `should` etc makes it more readable though 👍 




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());

Review comment:
       I find this use of streams awkward. I doubt performance is a problem, but it iterates over the groups twice and creates a bunch of sub-streams. I usually find it more straightforward to write a simple for loop.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+

Review comment:
       extra line




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {

Review comment:
       Could this be a static inner class instead?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {

Review comment:
       It is odd to have a class that is a `Util`. Most of the time that is used to name classes that carry static methods, rather than helpers. Is there a better name for this?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       At least on Parquet, differences in compression and encoding seem to be issues here. @aokolnychyi has more info but one of the hypothesis was that smaller files used dictionary encoding while larger files did not.




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+          .mode("append")
+          .save(table.name());

Review comment:
       That's true, we don't have required distribution and ordering in OSS Spark yet. Maybe, we can add it once 3.2 is out?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+
+    fileSetIDs.stream().map(id -> toID(table, id)).forEach(resultMap::remove);

Review comment:
       The `commitRewrite` methods are only called from tests. Since this has changed quite a bit, have we looked to see what is still needed?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));

Review comment:
       Added additional message here on commit failure with a ValidationException




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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -71,8 +71,9 @@
    * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
    * This will most likely be Action framework specific (Spark/Presto/Flink ....).
    *
+   * @param setId an identifier for this set of files

Review comment:
       Since we are rewriting files in groups (if I'm not mistaken), would it make more sense to refer to this as a `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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {

Review comment:
       This is only in the case that partialProgress is not enabled, in that case we either have 1 commit (this section of the code) so there is no need to stop rewrites since they have all finished




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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > > We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at.
   > 
   > Yes I am aware of this condition, but the delete files are applied for each file scan task anyway, it's just we cannot remove it because of the condition you described, and we have to call another action, do double work to fully remove the file. Conversely, say we have another action to only remove delete files, then we are reading those delete files anyway, and it also feels wasteful to me that we have to do another bin pack after deleting those files to make the files more optimized, and potentially cause more commit conflicts.
   
   They are only applied to the files we are looking at. It's not the reading of the delete files that is expensive, to determine if a delete file isn't needed we have to scan through *every* data file they may apply to. So the main pain here isn't that we have to read delete files more than once, but that to actually remove delete files you basically must do a full scan of the entire partition.
   
   For example
   
   Delete file A may touch Files A, B , C ,D
   We plan for files A and B
   After running binpack we Read A and B and write A' and B' we never touch C and D.
   Delete File A may now only touch C, and D 
   
   If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   made progress.
   
   
   It's only in the case that we run a full compaction that we know that we can remove delete files, and in that case removing delete files is basically free. We don't have to read anything and can just remove all delete files, no risk of conflicts since the files don't do anything.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Thanks everyone!


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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {

Review comment:
       Note, job desc is set in base 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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {

Review comment:
       Maybe I should just switch PartialProgress to a completely different code path
   may be easier to understand




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

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



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


[GitHub] [iceberg] rdblue merged pull request #2591: Spark: RewriteDatafilesAction V2

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


   


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {

Review comment:
       I think this is never null in `RewriteFileGroup` (which is correct) so we may drop this condition?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {

Review comment:
       Moved it closer to the execute methods




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+

Review comment:
       Also moving that new private commitOrClean below the abstract methods




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?

Review comment:
       Yep left over from the unified execute, i'll clean that up




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();

Review comment:
       I think this is a good change, implementing




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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();
+      cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+      Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+          .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+          .option(SparkReadOptions.FILE_OPEN_COST, "0")

Review comment:
       Thanks for the details explanation!




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {

Review comment:
       Currently the difficulty here is with our commit process, in both cases only the Action knows whether or not it has enough information to do the commit. We could put the commit api into the Strategy as well like how i've setup the action at the moment but that means changing the strategy API again and including more group tracking into the strategy.
   
   For example, in this case the Strategy has the rewriteCoord + manager but other implementations may not require this. A Strategy without a cache like this that just reads and writes files would need an api like
   
   ```java
   "commit(Files)"
   ```
   
   While this implementation would need
   ```java
   "commit(groupId)"
   ```
   
   This is why I had to add in the "groupId" to strategy for rewrite, so we would have some way of keeping
   track of the groupID outside of the strategy. If we wanted to make Strategy always responsible for being able to keep track of all files based on "groupId" then I think we would be ok. Then I would do an api like
   
   ```java
   
     // Writes new files from filesToRewrite and associates the result with groupID
     public void rewriteFiles(String groupID, List<FileScanTask> filesToRewrite)
     // Returns added files associated with groupID
     public Set<DataFiles> addedFiles(String groupID)
     // Returns file tasks which were rewritten in groupID
     public Set<DataFiles> rewrittenFiles(String groupID)
     
     // Performs the actual commit operation for the associated groupIDs, removes groupIds when complete
     public void commitGroupIDs(Set<String> groupID)
     // Removes all added files associated with groupID, removes groupIds when complete
     public void abortGroupIDs(Set<String> 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?

Review comment:
       Isn't `partialProgressEnabled` always true in this case?
    




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()
+                .setNameFormat("Committer-Service")
+                .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    rewriteService.shutdown();
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    if (results.size() == 0) {
+      LOG.error("{} is true but no rewrite commits succeeded. Check the logs to determine why the individual" +

Review comment:
       nit: We should add spaces at the end of these 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.
+   * This helps us in the case where our estimate for the task size is under the target size but the
+   * actual written file size is slightly larger.
+   * @return the target size plus one half of the distance between max and target
+   */
+  protected long writeMaxFileSize() {
+    return (long) (this.targetFileSize + ((this.maxFileSize - this.targetFileSize) * 0.5));

Review comment:
       nit: Do we need the explicit `this` here?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();

Review comment:
       Yeah, we could do a Thread.sleep instead. but we could do that

##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();

Review comment:
       Yeah, we could do a Thread.sleep instead.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSpark3.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.actions.rewrite.Spark3BinPackStrategy;
+import org.apache.spark.sql.SparkSession;
+
+public class RewriteDataFilesSpark3 extends BaseRewriteDataFilesSparkAction {
+  private FileRewriteCoordinator coordinator = FileRewriteCoordinator.get();
+
+  protected RewriteDataFilesSpark3(SparkSession spark, Table table) {
+    super(spark, table);
+  }
+
+  @Override
+  protected RewriteStrategy rewriteStrategy(Strategy type) {
+    switch (type) {
+      case BINPACK: return new Spark3BinPackStrategy(table(), spark());
+    }
+    throw new IllegalArgumentException(String.format(

Review comment:
       nit: What about moving `throw` to `default` in `switch`?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -71,8 +71,9 @@
    * Method which will rewrite files based on this particular RewriteStrategy's algorithm.
    * This will most likely be Action framework specific (Spark/Presto/Flink ....).
    *
+   * @param setId an identifier for this set of files
    * @param filesToRewrite a group of files to be rewritten together
    * @return a list of newly written files
    */
-  List<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite);
+  Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite);

Review comment:
       swapping everything to 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);

Review comment:
       typo: `interrupted`?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkFilesScan.java
##########
@@ -62,6 +63,9 @@
     long tableSplitSize = PropertyUtil.propertyAsLong(props, SPLIT_SIZE, SPLIT_SIZE_DEFAULT);
     this.splitSize = Spark3Util.propertyAsLong(options, SparkReadOptions.SPLIT_SIZE, tableSplitSize);
 
+    Preconditions.checkArgument(splitSize > 0,

Review comment:
       I think we better move this validation to TableScanUtil and also validate other args such as `lookback`.




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



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

Review comment:
       I think this should be `setOutputFiles` since this is modifying the group.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);
+    }
+
+    /**
+     * Returns all File groups which have been committed
+     */
+    public List<RewriteFileGroup> results() {
+      Preconditions.checkState(committerService.isShutdown(),
+          "Cannot get results from a service which has not been closed");
+      return committedRewrites;
+    }
+
+    @Override
+    public void close() {
+      Preconditions.checkState(running.compareAndSet(true, false),
+          "Cannot close already closed RewriteService");
+      LOG.info("Closing commit service for {}", table);
+      committerService.shutdown();
+
+      try {
+        // All rewrites have completed and all new files have been created, we are now waiting for the commit
+        // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+        // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+        // In either case this should take much less than 10 minutes to actually complete.
+        if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+          LOG.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean " +
+              "that changes were not successfully committed to the the Iceberg table.");
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);

Review comment:
       Thread.currentThread().interrupt?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > > We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at.
   > 
   > Yes I am aware of this condition, but the delete files are applied for each file scan task anyway, it's just we cannot remove it because of the condition you described, and we have to call another action, do double work to fully remove the file. Conversely, say we have another action to only remove delete files, then we are reading those delete files anyway, and it also feels wasteful to me that we have to do another bin pack after deleting those files to make the files more optimized, and potentially cause more commit conflicts.
   
   They are only applied to the files we are looking at. It's not the reading of the delete files that is expensive, to determine if a delete file isn't needed we have to scan through *every* data file they may apply to. After the bin pack we actually remove the files that are rewritten from the pool of files the Delete files may apply too but there is no way to check the rest without reading every file a delete file applies too. That's why it's completely independent, you don't actually do any work while rewriting files that benefits the delete file situation for other files.
   
   Delete file A may touch Files A, B , C ,D
   We plan for files A and B
   After running binpack we Read A and B and write A' and B' we never touch C and D.
   Delete File A may now only touch C, and D 


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

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.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 org.apache.iceberg.StructLike;
+
+public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo {

Review comment:
       Yes,  we will need the override `equals` and `hashCode` because I see we will use this `info` as keys in a [HashMap](https://github.com/apache/iceberg/pull/2591/files#diff-c04827ed079aa6cdede691f7ead4beaf30adc50f05da401a605215cdcd6dea4cR274).




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

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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       Nit: the name of this function and then the language used in the code seem sort of unrelated.  Is there a name for this that would more reflect the error message (and code)? Or perhaps a small doc comment might help to make this more clear to readers? Most things look great but I'm admittedly second guessing whether I understand why the function is named the way it 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));

Review comment:
       Switched to a simple foreach




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()

Review comment:
       The current one makes sense.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = filterAndGroupFiles(files, strategy);
+
+    try {
+      files.close();
+    } catch (IOException io) {
+      LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+    }
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+    );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = fileGroupsByPartition.entrySet().stream()

Review comment:
       I think by creating `toJobStream(fileGroupsByPartition)`, you should be able encapsulate the index vars above.




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

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



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


[GitHub] [iceberg] jackye1995 commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > We talked about this previously as a possible post-merge, post-delete, post-rewrite sort of thing. 
   
   Cool, that `cleanUnreferencedDeleteFiles()` was just a divergent thought, great that we already thought about it.
   
   > If file C for example is the correct size, and we never need to rewrite it, we never clean up those deletes so we still have to make another sort of action to clean up those files.
   
   Yes, that goes back to what I was thinking before, if we can have an option to force check the delete file and avoid filtering it out of the rewrite, then it should work.
   
   But I think I am starting to see where you are coming from. If this is done as a different action then we can save the write time if the file read does not contain any rows to delete in the delete file. To enable such a check in Spark, it cannot use the same code path that fully read all the rows and write it back. So it probably does not make sense to add delete functionality from that perspective. Thanks for the clarification!


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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;

Review comment:
       nit: I guess this will be renamed into `xxxRewrites` following the rename of the option.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);

Review comment:
       nit: I think the name of the var does not match the method name. We should probably align them.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );

Review comment:
       nit: I think `);` can be on the previous line where `collect` is.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);

Review comment:
       nit: `doExecuteWithPartialProgress`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();

Review comment:
       This method uses `infoListPair.first()` and `infoListPair.second()` quite a bit and I keep going back to see what each of those mean. Also, `infoListPair` isn't very descriptive and `Pair<FileGroupInfo, List<FileScanTask>>` is kind of bulky.
   
   While looking for a slightly more descriptive name, I saw it is also called a group in other places. What about creating a private helper class instead of using `Pair`? Also, what about creating temp vars that refer to the info and files? That should shorten other lines.
   
   ```
     private static class Group {
       private final FileGroupInfo groupInfo;
       private final List<FileScanTask> files;
   
       Group(FileGroupInfo groupInfo, List<FileScanTask> files) {
         this.groupInfo = groupInfo;
         this.files = files;
       }
   
       public FileGroupInfo info() {
         return groupInfo;
       }
   
       public List<FileScanTask> files() {
         return files;
       }
     }
   ```
   
   Then we can have `toGroupStream` that would construct a stream of groups.
   
   Then `rewriteFiles` can accept an instance of `Group`.
   

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()

Review comment:
       These vars are passed around in a lot of methods. What about creating a context object like we have in a few other places in Iceberg?
   
   I think something like this would simplify the code:
   
   ```
     private static class ExecutionContext {
       private final Map<StructLike, Integer> numGroupsByPartition;
       private final int totalGroupsCount;
       private final Map<StructLike, Integer> partitionIndexMap;
       private final AtomicInteger groupIndex;
   
       public ExecutionContext(Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
         this.numGroupsByPartition = fileGroupsByPartition.entrySet().stream()
             .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size()));;
         this.totalGroupsCount = numGroupsByPartition.values().stream()
             .reduce(Integer::sum)
             .orElse(0);
         this.partitionIndexMap = Maps.newConcurrentMap();
         this.groupIndex = new AtomicInteger(1);
       }
   
       public int totalGroupsCount() {
         return totalGroupsCount;
       }
   
       public int totalPartitionGroupsCount(StructLike partition) {
         return numGroupsByPartition.get(partition);
       }
   
       public int currentGlobalIndex() {
         return groupIndex.getAndIncrement();
       }
   
       public int currentPartitionIndex(StructLike partition) {
         return partitionIndexMap.merge(partition, 1, Integer::sum);
       }
     }
   ```
   
   Then our `execute` can be as simple as:
   
   ```
     @Override
     public Result execute() {
       validateOptions();
   
       Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
   
       if (fileGroupsByPartition.isEmpty()) {
         return new Result(Collections.emptyMap());
       }
   
       ExecutionContext ctx = new ExecutionContext(fileGroupsByPartition);
       Stream<Group> groupStream = toGroupStream(ctx, fileGroupsByPartition);
   
       if (partialProgressEnabled) {
         return doExecutePartialProgress(ctx, groupStream);
       } else {
         return doExecute(ctx, groupStream);
       }
     }
   ```

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);

Review comment:
       nit: can be just `int`?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);

Review comment:
       I think we are missing the set of group ids here.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATAFILES", desc),

Review comment:
       nit: `xxx-DATA-FILES`

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {

Review comment:
       nit: I think the order of methods in this class can be a little bit improved. For example, this one is not referenced by any of the adjacent methods so I don't know where it belongs or how it is used unless I open this in an IDE.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,

Review comment:
       nit: arg formatting

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATAFILES", desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      int totalGroups, Map<StructLike, Integer> groupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair -> rewriteFiles(infoListPair, totalGroups, groupsPerPartition, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      int totalGroups, Map<StructLike, Integer> groupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID); })
+        .run(infoListPair -> rewriteFiles(infoListPair, totalGroups, groupsPerPartition, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Stream<Pair<FileGroupInfo, List<FileScanTask>>> toJobStream(
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) {
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    return fileGroupsByPartition.entrySet().stream()
+        .flatMap(
+            e -> e.getValue().stream().map(tasks -> {
+              int myJobIndex = jobIndex.getAndIncrement();
+              int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+              String groupID = UUID.randomUUID().toString();
+              return Pair.of(new FileGroupInfo(groupID, myJobIndex, myPartIndex, e.getKey()), tasks);
+            }));
+  }
+
+  private void validateOptions() {
+
+    Set<String> validOptions = Sets.newHashSet(strategy.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 RewriteDatafiles or the strategy %s",
+        invalidKeys, strategy.name());
+
+    maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS_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);
+
+    Preconditions.checkArgument(maxConcurrentFileGroupActions >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_ACTIONS, maxConcurrentFileGroupActions);
+
+    Preconditions.checkArgument(!partialProgressEnabled || 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(FileGroupInfo fileGroupInfo, int totalGroups,
+      int numFilesToRewrite, int numFilesPerPartition, String strategyName) {

Review comment:
       Should `numFilesPerPartition` be `numGroupsPerPartition`?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.
+   * This helps us in the case where our estimate for the task size is under the target size but the
+   * actual written file size is slightly larger.
+   * @return the target size plus one half of the distance between max and target
+   */
+  protected long writeMaxFileSize() {
+    return (long) (this.targetFileSize + ((this.maxFileSize - this.targetFileSize) * 0.5));
+

Review comment:
       nit: extra line




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {

Review comment:
       Actually I ran into some worries here, I'm a little nervous about the pieces of this class that have state, like the table and executor service (and queue). I think we are safer if it's not static.




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

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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();
+      cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+      Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+          .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+          .option(SparkReadOptions.FILE_OPEN_COST, "0")

Review comment:
       Question for my own understanding: Why are you setting `FILE_OPEN_COST` to zero 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] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       Nit: the name of this function and then the language used in the code seem sort of unrelated.  Is there a name for this that would more reflect the error message (and code)? Or perhaps a small doc comment might help to make this more clear to readers? Most things look great but I'm admittedly second guessing whether I understand why the function is named the way it is.
   
   
   EDIT: Looking at the getOverlappingFiles function again, I still feel the name is a little confusing for me but no comment is needed.  I think my mind just hung up on the usage of `should` in this particular function name, but it's consistent with the other functions so I wouldn't personally change it. Feel free to close!




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -93,7 +96,7 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     return Collections.unmodifiableSet(rewrittenDataFiles);
   }
 
-  private Set<DataFile> fetchNewDataFiles(Table table, Set<String> fileSetIDs) {
+  public Set<DataFile> fetchNewDataFiles(Table table, Set<String> fileSetIDs) {

Review comment:
       I have to delete code out of FileRewriteCoordinator to do this since the function is used in CommitOrRewrite. I can do this in the followup pr?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of files to be rewritten by a RewriteAction and the new files which have been
+ * written by the action.
+ */
+public class RewriteFileGroup {
+  private final RewriteDataFiles.FileGroupInfo info;
+  private final List<FileScanTask> fileScanTasks;
+  private final int numInputFiles;
+
+  private Set<DataFile> outputFiles = Collections.emptySet();
+  private int numOutputFiles;
+
+  public RewriteFileGroup(RewriteDataFiles.FileGroupInfo info, List<FileScanTask> fileScanTasks) {
+    this.info = info;
+    this.fileScanTasks = fileScanTasks;
+    this.numInputFiles = fileScanTasks.size();
+  }
+
+  public int numInputFiles() {
+    return numInputFiles;
+  }
+
+  public StructLike partition() {
+    return info.partition();
+  }
+
+  public Integer globalIndex() {
+    return info.globalIndex();
+  }
+
+  public Integer partitionIndex() {
+    return info.partitionIndex();
+  }
+
+  public RewriteDataFiles.FileGroupInfo info() {
+    return info;
+  }
+
+  public List<FileScanTask> fileScans() {
+    return fileScanTasks;
+  }
+
+  public void outputFiles(Set<DataFile> files) {
+    numOutputFiles = files.size();
+    outputFiles = files;
+  }
+
+  public List<DataFile> rewrittenFiles() {

Review comment:
       Is there a particular reason why this is a list, not a set?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > Overall looks good to me. I would like to revisit the RewriteStrategy idea a bit. Because we are basically going to rewrite and remove all the delete files in this action along the way, this is what I see as the method for running major compaction.
   > 
   We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at. For example, we may determine File A should be read in conjunction with Delete File 1, and that it should be split into multiple files. We cannot remove DeleteFile 1 because it may also apply to Files B, and C which we didn't even consider.
   
   To actually determine delete files you must check the delete file against every valid file (not the other way around) and only if the delete file cannot be applied to any live files can you mark it as non-live (trying to avoid deleted deleted file).
   
   Because of that complexity we were hoping to move DeleteFiles compaction and cleanup to another action entirely. 


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       Okay, I just looked into Spark a bit more and I don't see anything keeping track of sessions created this way. My comment was based on an issue about [closing a session](http://apache-spark-developers-list.1001551.n3.nabble.com/Closing-a-SparkSession-stops-the-SparkContext-td26932.html). Looks like it's okay to just let them go away, I guess?




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {

Review comment:
       Renamed this to RewriteDataFilesCommitManager




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+

Review comment:
       Nit: I would expect the empty line before the `if` and comment rather than just inside.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);

Review comment:
       Yep, this is the amount of time we give the service after all rewrites have completed (Tasks.run blocks) for all commits to finish.
   
   So basically once all rewrite jobs have completed the remaining commits have 10 minutes to execute




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =
+        filesByPartition.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream().collect(Collectors.toMap(
+        e -> e.getKey(),
+        e -> e.getValue().size()
+    ));
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+    Integer totalPartitions = numGroupsPerPartition.keySet().size();
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream = fileGroupsByPartition.entrySet().stream().flatMap(
+        e -> e.getValue().stream().map(tasks -> {
+          int myJobIndex = jobIndex.getAndIncrement();
+          int myPartIndex = partitionIndex.merge(e.getKey(), 1, Integer::sum);
+          return Pair.of(new FileGroupInfo(UUID.randomUUID().toString(), myJobIndex, myPartIndex, e.getKey()), tasks);
+        })
+    );
+
+    int maxConcurrentFileGroupActions = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+        RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_ACTIONS_DEFAULT);
+
+    int maxCommits = PropertyUtil.propertyAsInt(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS,
+        RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    boolean partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(),
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED,
+        RewriteDataFiles.PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    int groupsPerCommit = partialProgressEnabled ? totalGroups / maxCommits : totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedCommits = new ConcurrentLinkedQueue<>();
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService commiterService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    commiterService.execute(() -> {
+      try {
+        while (stillRewriting.get() || completedRewrite.size() > groupsPerCommit) {
+          Thread.yield();
+          if (completedRewrite.size() > groupsPerCommit) {
+            // Gather a group of rewrites to do a commit
+            Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+            for (int i = 0; i < groupsPerCommit; i++) {
+              batch.add(completedRewrite.poll());
+            }
+            // Running commit on the group we gathered
+            try {
+              commitFileGroups(batch);
+            } catch (Exception e) {
+              if (!partialProgressEnabled) {
+                LOG.error("Failure during rewrite commit process, partial progress not enabled. Rethrowing", e);
+                throw e;
+              } else {
+                LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+              }
+            }
+            completedCommits.addAll(batch);
+          }
+        }
+        Set<String> finalBatch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+        while (completedRewrite.size() != 0) {
+          finalBatch.add(completedRewrite.poll());
+        }
+        if (finalBatch.size() != 0) {
+          // Final Commit
+          commitFileGroups(finalBatch);
+          completedCommits.addAll(finalBatch);
+        }
+      } catch (Exception e) {
+        if (!partialProgressEnabled) {
+          LOG.error("Cannot commit rewrite and partial progress not enabled, removing all completed work", e);
+          completedCommits.forEach(this::abortFileGroup);
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().setId);
+        });
+
+    if (partialProgressEnabled) {
+      rewriteTaskBuilder = rewriteTaskBuilder.suppressFailureWhenFinished();
+    } else {
+      rewriteTaskBuilder = rewriteTaskBuilder.stopOnFailure();
+    }
+
+    rewriteTaskBuilder
+        .run(i -> {
+          String setId = i.first().setId();
+          Set<DataFile> addedFiles = strategy.rewriteFiles(setId, i.second());
+          completedRewrite.offer(setId);
+          results.put(setId, Pair.of(i.first(), new FileGroupRewriteResult(addedFiles.size(), i.second().size())));
+        });
+    stillRewriting.set(false);
+    commiterService.shutdown();
+
+    try {
+      commiterService.awaitTermination(10, TimeUnit.MINUTES);

Review comment:
       Does it mean we give 10 minutes for the committer thread to gracefully shut down?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+

Review comment:
       I think we need to remove redundant logic from this class now. Can be done in a separate PR to simplify the review.
   We should probably rename it too as it is no longer a commit coordinator.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {

Review comment:
       I want to keep this as an object mostly so I can do tests on failures within this code. It's much easier than if these are all static method calls




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Table table;
+  private Expression filter = Expressions.alwaysTrue();
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+
+    CloseableIterable<FileScanTask> files = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files).collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition =

Review comment:
       Yep was planning on having grouping and filtering separate just haven't got to it yet




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Map;
+
+public class BaseRewriteDataFilesResult implements RewriteDataFiles.Result {
+  private final Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap;
+
+  public BaseRewriteDataFilesResult(
+      Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap) {
+    this.resultMap = resultMap;
+  }
+
+  @Override
+  public Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap() {

Review comment:
       This will be also way shorter with direct imports.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      try {

Review comment:
       I think we probably would wrap this whole thing into a new api, and make it a transient member of a strategy or something like that? Otherwise I think we have serializability errors ...




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = filterAndGroupFiles(files, strategy);
+
+    try {
+      files.close();
+    } catch (IOException io) {
+      throw new UncheckedIOException("Cannot properly close file iterable while planning for rewrite", io);

Review comment:
       Sure, we could just log an error and keep going




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {

Review comment:
       Why would there be no added files in a group? This seems like a suspicious case where something else has gone wrong.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+

Review comment:
       Yeah i'll do another pr after this which prunes 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] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



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

Review comment:
       Are all these methods needed? It seems like this may only need `info` and `asResult`?




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

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

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.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 org.apache.iceberg.StructLike;
+
+public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo {
+  private final int globalIndex;
+  private final int partitionIndex;
+  private final StructLike partition;
+
+  public BaseRewriteDataFilesFileGroupInfo(int globalIndex, int partitionIndex, StructLike partition) {
+    this.globalIndex = globalIndex;
+    this.partitionIndex = partitionIndex;
+    this.partition = partition;
+  }
+
+  @Override
+  public int globalIndex() {
+    return globalIndex;
+  }
+
+  @Override
+  public int partitionIndex() {
+    return partitionIndex;
+  }
+
+  @Override
+  public StructLike partition() {
+    return partition;
+  }
+
+  @Override
+  public String toString() {
+    return "FileGroupInfo{" +

Review comment:
       Nit:   It will be more efficient if we follow the similar `toString` approach from [BaseFile.java](https://github.com/apache/iceberg/blob/bc332b60789ce6f9aad822f79770c68a828965d9/core/src/main/java/org/apache/iceberg/BaseFile.java#L441).




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {

Review comment:
       Of course, trying hard to get everything in :)




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {

Review comment:
       Changed this to a check state since as @rdblue noted this should never be called on a file group which hasn't already successfully gone through rewriting.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);

Review comment:
       I think this should be debug rather than info. We don't need to know when groups are handed off in normal operation, 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();
+      cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+      Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+          .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+          .option(SparkReadOptions.FILE_OPEN_COST, "0")

Review comment:
       So normally we are trying to weight the cost of opening a new file vs the cost of starting a new spark task. Opening files is expensive and sometimes it makes more sense to just start another task in parallel rather than waiting to open another file in serial within the same task. Imagine the case of many small files, if we pretend they are slightly larger with this value then we will bias our splits to having less small files in them which helps use make more spark tasks even if they read less data.
   
   In this case we aren't optimizing for read performance, instead we are trying to make each Spark task as close to our target size as possible regardless of how many small files it takes. Here our goal is to write files of the correct size so the number of input files doesn't really matter. To do this we set the open cost to 0.




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

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

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



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


[GitHub] [iceberg] jackye1995 commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at.
   
   Yes I am aware of this condition, but the delete files are applied for each file scan task anyway, it's just we cannot remove it because of the condition you described, and we have to call another action, do double work to fully remove the file. Conversely, say we have another action to only remove delete files, then we are reading those delete files anyway, and it also feels wasteful to me that we have to do another bin pack after deleting those files to make the files more optimized, and potentially cause more commit conflicts.
   
   I understand there is a good separation of concern if we do them as 2 different actions. But when I try to imagine what the compaction API looks like, it seems that I just need a different `selectFilesToRewrite` implementation of the rewrite strategy, and other things can mostly be reused with just a few small branching logic.
   
   So instead of having another totally different action that removes delete file, the major compaction can potentially be done as just an extension to the existing strategy or a replacement to run a different strategy. For example, we can extend the current bin pack strategy with: if there are delete files in a file scan task, then the data file must be included for rewriting. We can also plug in a strategy that try to select all data files based on a certain delete file threshold, etc.
   
   We can get more clever about that as we evolve, but the general thought I have is that having data file rewriting and delete file compaction as one base action with different strategies to satisfy different use cases seems to be a more efficient and flexible way to go.
   


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       I'm just a bit worried about creating session objects that never get cleaned up. This is a leak for anything calling this from a service, 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java
##########
@@ -50,4 +50,7 @@ private SparkWriteOptions() {
 
   // File scan task set ID that indicates which files must be replaced
   public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id";
+
+  // Partition spec ID to use when writing new files
+  public static final String SPEC_ID = "spec-id";

Review comment:
       I need to actually delete this, I had an implementation for the spark writer using this property but it did not work. I"ll remove this for now




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -27,7 +27,7 @@
 import org.apache.iceberg.FileScanTask;
 import org.apache.iceberg.Table;
 
-interface RewriteStrategy extends Serializable {
+public interface RewriteStrategy extends Serializable {

Review comment:
       One of the things I was having trouble with here is that we have iceberg.actions and spark.iceberg.actions so I can't keep this package private and use what I think is the correct package for the implementations




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+        );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = toJobStream(fileGroupsByPartition);
+
+    if (partialProgressEnabled) {
+      return doExecutePartialProgress(groupStream, totalGroups, numGroupsPerPartition);
+    } else {
+      return doExecute(groupStream, totalGroups, numGroupsPerPartition);
+    }
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();

Review comment:
       This method uses `infoListPair.first()` and `infoListPair.second()` quite a bit and I keep going back to see what each of those mean. Also, `infoListPair` isn't very descriptive and `Pair<FileGroupInfo, List<FileScanTask>>` is kind of bulky.
   
   While looking for a slightly more descriptive name, I saw it is also called a group in other places. What about creating a private helper class instead of using `Pair`? Also, what about creating temp vars that refer to the info and files? That should shorten other lines.
   
   ```
     private static class Group {
       private final FileGroupInfo groupInfo;
       private final List<FileScanTask> files;
   
       Group(FileGroupInfo groupInfo, List<FileScanTask> files) {
         this.groupInfo = groupInfo;
         this.files = files;
       }
   
       public FileGroupInfo info() {
         return groupInfo;
       }
   
       public List<FileScanTask> files() {
         return files;
       }
     }
   ```
   
   Then we can have `toGroupStream` that would construct a stream of groups instead of having `toJobStream` that returns `Stream<Pair<FileGroupInfo, List<FileScanTask>>>`.
   
   Then `rewriteFiles` can accept an instance of `Group`.
   




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/rewrite/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions.rewrite;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String setId, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, setId, filesToRewrite);
+
+    // read and pack original 4 files into 2 splits
+    Dataset<Row> scanDF = spark.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, setId)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, setId)
+          .mode("append")
+          .save(table.name());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+    return rewriteCoordinator.fetchNewDataFiles(table, ImmutableSet.of(setId));

Review comment:
       Hmm I think that's true, it does mean i have to handle cleanup slightly differently in the action itself. The case where we don't have partial progress enabled but do have multiple file groups being committed can lead to a situation where several sets of files that have been successfully rewritten but since the commit is invalid we need to clear them out.
   
   It's late so we can probably talk about this tomorrow after I slept a bit.




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but

Review comment:
       I think the first sentence should be little bit more clear that we estimate the tasks correctly and we hope the generated file (not task) will be less than or equal to our target file size...




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);

Review comment:
       I think I could do warn, but "info" implies expected to me, and this is not an expected behavior. Imho




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()

Review comment:
       This is the style used in our other executor creation?
   I'm not sure what the preferred formatting is here - for example from ThreadPools.java
   
   ```  private static final ExecutorService WORKER_POOL = MoreExecutors.getExitingExecutorService(
         (ThreadPoolExecutor) Executors.newFixedThreadPool(
             WORKER_THREAD_POOL_SIZE,
             new ThreadFactoryBuilder()
                 .setDaemon(true)
                 .setNameFormat("iceberg-worker-pool-%d")
                 .build()));
   ```




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {

Review comment:
       I think `xxxUtil` usually means the class is stateless. Shall we come up with a different name?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {

Review comment:
       Or maybe just pass an explicit table to methods if we want to keep it as a util 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] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;

Review comment:
       nit: why not return the result from `rewriteCoordinator.fetchNewDataFiles` 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Map;
+
+public class BaseRewriteDataFilesResult implements RewriteDataFiles.Result {
+  private final Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap;
+
+  public BaseRewriteDataFilesResult(
+      Map<RewriteDataFiles.FileGroupInfo, RewriteDataFiles.FileGroupRewriteResult> resultMap) {

Review comment:
       Actually I don't need statics here, woops




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+          .mode("append")
+          .save(table.name());

Review comment:
       I didn't think had this implemented yet? But I can add in a sortWithinPartitions here




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       We have seen this quite a lot when tiny Parquet files are compacted into larger ones as it changes the encoding on many columns. In most cases, the actual file size is bigger than what we estimated. 
   
   I am not sure about Avro. Cases where the estimation is precise enough should work as expected. The main cases we are trying to avoid is splitting 514 MB into 512 and 2 MB files and writing 1 GB files when the target is 512 MB.
   
   The ideal solution is to know how much the remaining rows are going to cost us. 




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {

Review comment:
       Yeah this would mean that the class has been passed incorrectly. FileGroup.addedFiles should always be set of abort is called 




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()
+                .setNameFormat("Committer-Service")
+                .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+          abortFileGroup(fileGroup.groupID());
+        })
+        .run(infoListPair -> rewriteFiles(ctx, infoListPair, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    rewriteService.shutdown();
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);

Review comment:
       I think a comment would help other folks to understand 10 mins. I suppose we leave 10 mins once the rewrite is done for remaining commits.
   
   We could also check the return value of `awaitTermination` and log a warning if we hit the timeout.




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);
+    }
+
+    /**
+     * Returns all File groups which have been committed
+     */
+    public List<RewriteFileGroup> results() {
+      Preconditions.checkState(committerService.isShutdown(),
+          "Cannot get results from a service which has not been closed");
+      return committedRewrites;
+    }
+
+    @Override
+    public void close() {
+      Preconditions.checkState(running.compareAndSet(true, false),
+          "Cannot close already closed RewriteService");
+      LOG.info("Closing commit service for {}", table);
+      committerService.shutdown();
+
+      try {
+        // All rewrites have completed and all new files have been created, we are now waiting for the commit
+        // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should
+        // have been occurring simultaneously with rewrites, if not there should be only a single commit operation.
+        // In either case this should take much less than 10 minutes to actually complete.
+        if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) {
+          LOG.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean " +
+              "that changes were not successfully committed to the the Iceberg table.");
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e);
+      }
+      Preconditions.checkState(completedRewrites.isEmpty(), "File groups offered after service was closed, " +
+          "they were not successfully committed.");

Review comment:
       Nit: I think it's better to move the string to the next line and avoid breaking it across 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] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitManager.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitManager(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = Sets.newHashSet();
+    Set<DataFile> addedDataFiles = Sets.newHashSet();
+    for (RewriteFileGroup group : fileGroups) {
+      rewrittenDataFiles = Sets.union(rewrittenDataFiles, group.rewrittenFiles());
+      addedDataFiles = Sets.union(addedDataFiles, group.addedFiles());
+    }
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, addedDataFiles)
+        .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(RewriteFileGroup fileGroup) {
+    Preconditions.checkState(fileGroup.addedFiles() != null,
+        "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      rewriteGroups.forEach(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private final AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          try {
+            if (completedRewrites.size() == 0) {
+              // Give other threads a chance to make progress
+              Thread.sleep(100);
+            }
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException("Interrupted while processing commits", e);
+          }
+
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.debug("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);

Review comment:
       I think there's a race condition that could cause a group to get ignored:
   * One thread enters this method and `running.get()` is true, but is then paused before adding the group.
   * Another thread calls close, setting `running` to false.
   * The service thread consumes the rest of `completedRewrites` and commits
   
   I don't think this is likely to happen given the current use of this class. Plus, the commit in the service thread would take forever and the original thread calling `offer` would almost certainly run. So maybe it isn't a real problem, but it seems like it would be a simple fix to use `synchronized` block to guard `completedRewrites`.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       This uncertainty is why I don't usually create new sessions. But we'll see if it actually becomes a problem.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();

Review comment:
       I think we currently don't remove partitions that end up getting 0 file groups after filtering. 




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.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 org.apache.iceberg.StructLike;
+
+public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo {

Review comment:
       I assume this is no longer used in maps?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite)))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()
+          .format("iceberg")
+          .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID)
+          .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+          .mode("append")
+          .save(table.name());

Review comment:
       Thoughts, @rdblue?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.rewrittenFiles().stream()).collect(Collectors.toSet());
+
+    Set<DataFile> newDataFiles = fileGroups.stream()
+        .flatMap(fileGroup -> fileGroup.addedFiles().stream()).collect(Collectors.toSet());
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, newDataFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param fileGroup fileSet to clean
+   */
+  public void abortFileGroup(RewriteFileGroup fileGroup) {
+    if (fileGroup.addedFiles() == null) {
+      // Nothing to clean
+      return;
+    }
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      Tasks.foreach(rewriteGroups)
+          .suppressFailureWhenFinished()
+          .run(group -> abortFileGroup(group));
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+          .setNameFormat("Committer-Service")
+          .build());
+
+      completedRewrites = Queues.newConcurrentLinkedQueue();
+      committedRewrites = Lists.newArrayList();
+    }
+
+    /**
+     * Starts a single threaded executor service for handling file group commits.
+     */
+    public void start() {
+      Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started");
+      LOG.info("Starting commit service for {}", table);
+      // Partial progress commit service
+      committerService.execute(() -> {
+        while (running.get() || completedRewrites.size() > 0) {
+          Thread.yield();
+          // Either we have a full commit group, or we have completed writing and need to commit what is left over
+          if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) {
+
+            Set<RewriteFileGroup> batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit);
+            for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) {
+              batch.add(completedRewrites.poll());
+            }
+
+            try {
+              commitOrClean(batch);
+              committedRewrites.addAll(batch);
+            } catch (Exception e) {
+              LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+            }
+          }
+        }
+      });
+    }
+
+    /**
+     * Places a file group in the queue to be asynchronously committed either when the queue has enough elements
+     * to do a batch of size {@link #rewritesPerCommit} or the service has been closed.
+     * @param group file group to eventually be committed
+     */
+    public void offer(RewriteFileGroup group) {
+      LOG.info("Offered to commit service: {}", group);
+      Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed");
+      completedRewrites.add(group);
+    }
+
+    /**
+     * Returns all File groups which have been committed
+     */
+    public List<RewriteFileGroup> results() {
+      Preconditions.checkState(committerService.isShutdown(),
+          "Cannot get results from a service which has not been closed");
+      return committedRewrites;
+    }
+
+    @Override
+    public void close() {
+      Preconditions.checkState(running.compareAndSet(true, false),

Review comment:
       nit: This class implements `Closeable` that requires `close` to be idempotent and it should be fine to call it multiple times. 




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

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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java
##########
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.File;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.actions.ActionsProvider;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteDataFiles.Result;
+import org.apache.iceberg.actions.RewriteDataFilesCommitUtil;
+import org.apache.iceberg.actions.RewriteFileGroup;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+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.Iterables;
+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.Streams;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+
+public abstract class TestNewRewriteDataFilesAction extends SparkTestBase {
+
+  protected abstract ActionsProvider actions();
+  protected abstract Set<String> cacheContents(Table table);
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+  }
+
+  private RewriteDataFiles basicRewrite(Table table) {
+    // Always compact regardless of input files
+    table.refresh();
+    return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1");
+  }
+
+  @Test
+  public void testEmptyTable() {
+    PartitionSpec spec = PartitionSpec.unpartitioned();
+    Map<String, String> options = Maps.newHashMap();
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    Assert.assertNull("Table must be empty", table.currentSnapshot());
+
+    basicRewrite(table).execute();
+
+    Assert.assertNull("Table must stay empty", table.currentSnapshot());
+  }
+
+  @Test
+  public void testBinPackUnpartitionedTable() {
+    Table table = createTable(4);
+    shouldHaveFiles(table, 4);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 1);
+    List<Object[]> actual = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actual);
+  }
+
+  @Test
+  public void testBinPackPartitionedTable() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+
+    Result result = basicRewrite(table).execute();
+    Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 4);
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackWithFilter() {
+    Table table = createTablePartitioned(4, 2);
+    shouldHaveFiles(table, 8);
+    List<Object[]> expectedRecords = currentData();
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c1", 1))
+        .filter(Expressions.startsWith("c2", "foo"))
+        .execute();
+
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 7);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testRewriteLargeTableHasResiduals() {
+    PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build();
+    Map<String, String> options = Maps.newHashMap();
+    options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100");
+    Table table = TABLES.create(SCHEMA, spec, options, tableLocation);
+
+    // all records belong to the same partition
+    List<ThreeColumnRecord> records = Lists.newArrayList();
+    for (int i = 0; i < 100; i++) {
+      records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4)));
+    }
+    Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class);
+    writeDF(df);
+
+    List<Object[]> expectedRecords = currentData();
+
+    table.refresh();
+
+    CloseableIterable<FileScanTask> tasks = table.newScan()
+        .ignoreResiduals()
+        .filter(Expressions.equal("c3", "0"))
+        .planFiles();
+    for (FileScanTask task : tasks) {
+      Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual());
+    }
+
+    shouldHaveFiles(table, 2);
+
+    Result result = basicRewrite(table)
+        .filter(Expressions.equal("c3", "0"))
+        .execute();
+    Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount());
+
+    List<Object[]> actualRecords = currentData();
+
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackSplitLargeFile() {
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    List<Object[]> expectedRecords = currentData();
+    long targetSize = testDataSize(table) / 2;
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize))
+        .execute();
+
+    Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount());
+    Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 2);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testBinPackCombineMixedFiles() {
+    // One file too big
+    Table table = createTable(1);
+    shouldHaveFiles(table, 1);
+
+    // Two files too small
+    writeRecords(1, 100);
+    writeRecords(1, 100);
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> expectedRecords = currentData();
+
+    int targetSize = averageFileSize(table);
+
+    Result result = basicRewrite(table)
+        .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize))
+        .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100))
+        .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100))
+        .execute();
+
+    Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount());
+    // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files
+    Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount());
+
+    shouldHaveFiles(table, 3);
+
+    List<Object[]> actualRecords = currentData();
+    assertEquals("Rows must match", expectedRecords, actualRecords);
+  }
+
+  @Test
+  public void testPartialProgressEnabled() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    shouldHaveSnapshots(table, 11);
+    shouldHaveACleanCache(table);
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+  }
+
+  @Test
+  public void testMultipleGroups() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(BinPackStrategy.MIN_INPUT_FILES, "1")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressMaxCommits() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    // Perform a rewrite but only allow 2 files to be compacted at a time
+    RewriteDataFiles.Result result =
+        basicRewrite(table)
+            .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3")
+            .execute();
+
+    Assert.assertEquals("Should have 10 fileGroups", result.resultMap().keySet().size(), 10);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 4);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testSingleCommitWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100));
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // Fail to commit
+    doThrow(new RuntimeException("Commit Failure"))
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelSingleCommitWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class,
+        () -> spyRewrite.execute());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 1);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithRewriteFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite);
+
+    // Fail groups 1, 3, and 7 during rewrite
+    GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7);
+    doThrow(new RuntimeException("Rewrite Failed"))
+        .when(spyRewrite)
+        .rewriteFiles(any(), argThat(failGroup));
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    Assert.assertEquals("Should have 7 fileGroups", result.resultMap().keySet().size(), 7);
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2.
+    // removing 3 groups leaves us with only 2 new commits, 4 and 3
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testParallelPartialProgressWithCommitFailure() {
+    Table table = createTable(20);
+    int fileSize = averageFileSize(table);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction realRewrite =
+        (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction)
+            basicRewrite(table)
+                .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100))
+                .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+                .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3");
+
+    BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    // First and Third commits work, second does not
+    doCallRealMethod()
+        .doThrow(new RuntimeException("Commit Failed"))
+        .doCallRealMethod()
+        .when(util)
+        .commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyRewrite)
+        .commitUtil();
+
+    RewriteDataFiles.Result result = spyRewrite.execute();
+
+    // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted
+    Assert.assertEquals("Should have 6 fileGroups", 6, result.resultMap().keySet().size());
+
+    table.refresh();
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    // Only 2 new commits because we broke one
+    shouldHaveSnapshots(table, 3);
+    shouldHaveNoOrphans(table);
+    shouldHaveACleanCache(table);
+  }
+
+  @Test
+  public void testInvalidOptions() {
+    Table table = createTable(20);
+
+    AssertHelpers.assertThrows("No negative values for partial progress max commits",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true")
+            .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No negative values for max concurrent groups",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5")
+            .execute());
+
+    AssertHelpers.assertThrows("No unknown options allowed",
+        IllegalArgumentException.class,
+        () -> basicRewrite(table)
+            .option("foobarity", "-5")
+            .execute());
+  }
+
+  @Test
+  public void testCommitStateUnknownException() {
+    Table table = createTable(20);
+    shouldHaveFiles(table, 20);
+
+    List<Object[]> originalData = currentData();
+
+    BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table);
+    BaseRewriteDataFilesSparkAction spyAction = spy(action);
+    RewriteDataFilesCommitUtil util = spy(new RewriteDataFilesCommitUtil(table));
+
+    doAnswer(invocationOnMock -> {
+      invocationOnMock.callRealMethod();
+      throw new CommitStateUnknownException(new RuntimeException("Unknown State"));
+    }).when(util).commitFileGroups(any());
+
+    doReturn(util)
+        .when(spyAction)
+        .commitUtil();
+
+    AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception",
+        CommitStateUnknownException.class, () -> spyAction.execute());
+
+    List<Object[]> postRewriteData = currentData();
+    assertEquals("We shouldn't have changed the data", originalData, postRewriteData);
+
+    shouldHaveSnapshots(table, 2); // Commit actually Succeeded
+  }
+
+  protected List<Object[]> currentData() {
+    return rowsToJava(spark.read().format("iceberg").load(tableLocation)
+        .sort("c1", "c2", "c3")
+        .collectAsList());
+  }
+
+  protected long testDataSize(Table table) {
+    return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum();
+  }
+
+  protected void shouldHaveFiles(Table table, int numExpected) {
+    table.refresh();
+    int numFiles = Iterables.size(table.newScan().planFiles());
+    Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles);
+  }
+
+  protected void shouldHaveSnapshots(Table table, int expectedSnapshots) {
+    table.refresh();
+    int actualSnapshots = Iterables.size(table.snapshots());
+    Assert.assertEquals("Table did not have the expected number of snapshots",
+        expectedSnapshots, actualSnapshots);
+  }
+
+  protected void shouldHaveNoOrphans(Table table) {
+    Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(),
+        actions().removeOrphanFiles(table)
+            .olderThan(System.currentTimeMillis())
+            .execute()
+            .orphanFileLocations());
+  }
+
+  protected void shouldHaveACleanCache(Table table) {
+    Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(),
+        cacheContents(table));
+  }
+
+  protected <T> void shouldHaveLastCommitSorted(Table table, String column) {

Review comment:
       I should clarify, more my confusion was less with `should` and more that it was `shouldHaveLastCommitSorted`, but the assertion is `Found overlapping files`.
   
   I guess I would have expected `shouldHaveNoOverlappingFiles`.
   
   I agree tha the `should` etc makes it more readable though 👍 




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitManager.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitManager(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {
+    Set<DataFile> rewrittenDataFiles = Sets.newHashSet();
+    Set<DataFile> addedDataFiles = Sets.newHashSet();
+    for (RewriteFileGroup group : fileGroups) {
+      rewrittenDataFiles = Sets.union(rewrittenDataFiles, group.rewrittenFiles());
+      addedDataFiles = Sets.union(addedDataFiles, group.addedFiles());
+    }
+
+    table.newRewrite()
+        .rewriteFiles(rewrittenDataFiles, addedDataFiles)
+        .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(RewriteFileGroup fileGroup) {
+    Preconditions.checkState(fileGroup.addedFiles() != null,
+        "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedFiles())
+        .noRetry()
+        .suppressFailureWhenFinished()
+        .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc))
+        .run(dataFile -> table.io().deleteFile(dataFile.path().toString()));
+  }
+
+  public void commitOrClean(Set<RewriteFileGroup> rewriteGroups) {
+    try {
+      commitFileGroups(rewriteGroups);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewriteGroups, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e);
+      rewriteGroups.forEach(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * An async service which allows for committing multiple file groups as their rewrites complete. The service also
+   * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not
+   * be offered.
+   * @param rewritesPerCommit number of file groups to include in a commit
+   * @return the service for handling commits
+   */
+  public CommitService service(int rewritesPerCommit) {
+    return new CommitService(rewritesPerCommit);
+  }
+
+  public class CommitService implements Closeable {
+    private final ExecutorService committerService;
+    private final ConcurrentLinkedQueue<RewriteFileGroup> completedRewrites;
+    private final List<RewriteFileGroup> committedRewrites;
+    private final int rewritesPerCommit;
+    private final AtomicBoolean running = new AtomicBoolean(false);
+
+    CommitService(int rewritesPerCommit) {
+      LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit);
+      this.rewritesPerCommit = rewritesPerCommit;
+
+      committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       Minor: assignments to instance state without `this.` prefix.




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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableIterable.java
##########
@@ -26,9 +26,11 @@
 import java.util.NoSuchElementException;
 import java.util.function.Function;
 import java.util.function.Predicate;
+import java.util.stream.Stream;

Review comment:
       Yeah not sure how this happened




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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   I agree can handle rewriting partitioning in a follow-up PR as this one is large enough.


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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Split Out Split Planning Validation - https://github.com/apache/iceberg/pull/2759 


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

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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Non-Spark Changes - https://github.com/apache/iceberg/pull/2760


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -162,4 +163,27 @@ private void validateOptions() {
         "Cannot set %s is less than 1. All values less than 1 have the same effect as 1. %d < 1",
         MIN_INPUT_FILES, minInputFiles);
   }
+
+  protected long targetFileSize() {
+    return this.targetFileSize;
+  }
+
+  /**
+   * Ideally every Spark Task that is generated will be less than or equal to our target size but
+   * in practice this is not the case. When we actually write our files, they may exceed the target
+   * size and end up being split. This would end up producing 2 files out of one task, one target sized
+   * and one very small file. Since the output file can vary in size, it is better to
+   * use a slightly larger (but still within threshold) size for actually writing the tasks out.

Review comment:
       We have seen this quite a lot when tiny Parquet files are compacted into larger ones as it changes the encoding on many columns. In most cases, the actual file size is bigger than what we estimated. 
   
   I am not sure about Avro. Cases where the estimation is precise enough should work as expected. The main cases we are trying to avoid is splitting 514 MB into 512 and 2 MB files and writing 1 GB files when the target is 512 MB.
   
   The ideal solution is to know how much the remaining rows are going to cost us but we don't have that implemented yet. 




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

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



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


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java
##########
@@ -50,4 +50,7 @@ private SparkWriteOptions() {
 
   // File scan task set ID that indicates which files must be replaced
   public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id";
+
+  // Partition spec ID to use when writing new files
+  public static final String SPEC_ID = "spec-id";

Review comment:
       should we make the constant from `PartitionSpecParser` public? then we don't have define it here as it is a more general constant than spark




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

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



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


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > > We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at.
   > 
   > Yes I am aware of this condition, but the delete files are applied for each file scan task anyway, it's just we cannot remove it because of the condition you described, and we have to call another action, do double work to fully remove the file. Conversely, say we have another action to only remove delete files, then we are reading those delete files anyway, and it also feels wasteful to me that we have to do another bin pack after deleting those files to make the files more optimized, and potentially cause more commit conflicts.
   
   They are only applied to the files we are looking at. It's not the reading of the delete files that is expensive, to determine if a delete file isn't needed we have to scan through *every* data file they may apply to. So the main pain here isn't that we have to read delete files more than once, but that to actually remove delete files you basically must do a full scan of the entire partition.
   
   For example
   
   Delete file A may touch Files A, B , C ,D
   We plan for files A and B
   After running binpack we Read A and B and write A' and B' we never touch C and D.
   Delete File A may now only touch C, and D 
   
   If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   made progress.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = filterAndGroupFiles(files, strategy);
+
+    try {
+      files.close();
+    } catch (IOException io) {
+      LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+    }
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())
+    );
+
+    Integer totalGroups = numGroupsPerPartition.values().stream().reduce(Integer::sum).orElse(0);
+
+    if (totalGroups == 0) {
+      return new Result(Collections.emptyMap());
+    }
+
+    Map<StructLike, Integer> partitionIndex = new HashMap<>();
+    AtomicInteger jobIndex = new AtomicInteger(1);
+
+    // Todo Check if we need to randomize the order in which we do jobs, instead of being partition centric
+    Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream = fileGroupsByPartition.entrySet().stream()

Review comment:
       Also moving execute up to be with the other public overrides




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

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



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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(

Review comment:
       nit: unnecessary newline

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {

Review comment:
       nit: this method is not used anywhere else and only 3 lines, I think we can just put it in the planFIleGroups method

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()

Review comment:
       Why not add some configurable retry for each file group?

##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -78,6 +78,7 @@
   private long maxFileSize;
   private long targetFileSize;
   private long maxGroupSize;
+  private long specId;

Review comment:
       specId is never assigned

##########
File path: core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java
##########
@@ -42,7 +42,7 @@
  * more files than {@link MIN_INPUT_FILES} or would produce at least one file of

Review comment:
       for all java doc to reference variables, you need `{@link #xxxxx}`, such as `{@link #MIN_INPUT_FILES}`

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);

Review comment:
       nit: info()

##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSpark3Action.java
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Set;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BaseRewriteDataFilesSpark3Action extends BaseRewriteDataFilesSparkAction {
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSpark3Action.class);
+  private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get();
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+
+  protected BaseRewriteDataFilesSpark3Action(SparkSession spark, Table table) {
+    super(spark, table);
+  }
+
+  @Override
+  protected RewriteStrategy defaultStrategy() {
+    return new Spark3BinPackStrategy(table(), spark());

Review comment:
       We are setting the bin pack strategy as the default strategy, but there is not really a way to overwrite it. So I think either we should call this action `BinPackDataFilesSpark3Action`, or we should load this by strategy name from options. 
   
   And either way, this does not sound like a `default` to me anymore, should we just call this method `strategy()`?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   > Overall looks good to me. I would like to revisit the RewriteStrategy idea a bit. Because we are basically going to rewrite and remove all the delete files in this action along the way, this is what I see as the method for running major compaction.
   > 
   We cannot and are not deleting delete files in this action because it's actually much more difficult to find out which delete files are no longer in use than just checking which ones are referred to by the FileScanTasks for the files we are looking at. For example, we may determine File A should be read in conjunction with Delete File 1, and that it should be split into multiple files. We cannot remove DeleteFile 1 because it may also apply to Files B, and C which we didn't even look .
   
   To actually determine delete files you must check the delete file against every valid file (not the other way around) and only if the delete file cannot be applied to any live files can you mark it as non-live (trying to avoid deleted deleted file).
   
   Because of that complexity we were hoping to move DeleteFiles compaction and cleanup to another action entirely. 


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   Thanks for discussing it with me, I definitely want to make this efficient and I know some of our future plans haven’t been documented yet. On thing I really want to finish is distributed planning where no machine ever gets the full scan plan.
   
   Sent from my iPhone
   
   > On May 21, 2021, at 12:11 AM, Jack Ye ***@***.***> wrote:
   > 
   > 
   > We talked about this previously as a possible post-merge, post-delete, post-rewrite sort of thing.
   > 
   > Cool, that cleanUnreferencedDeleteFiles() was just a divergent thought, great that we already thought about it.
   > 
   > If file C for example is the correct size, and we never need to rewrite it, we never clean up those deletes so we still have to make another sort of action to clean up those files.
   > 
   > Yes, that goes back to what I was thinking before, if we can have an option to force check the delete file and avoid filtering it out of the rewrite, then it should work.
   > 
   > But I think I am starting to see where you are coming from. If this is done as a different action then we can save the write time if the file read does not contain any rows to delete in the delete file. To enable such a check in Spark, it cannot use the same code path that fully read all the rows and write it back. So it probably does not make sense to add delete functionality from that perspective. Thanks for the clarification!
   > 
   > —
   > You are receiving this because you authored the thread.
   > Reply to this email directly, view it on GitHub, or unsubscribe.
   


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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #2591: Spark: RewriteDatafilesAction V2

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


   I know this is a bit behind on what Russell has locally but a few high-level points:
   
   - Shall we consider a separate quick PR for tiny updates to the existing code? Like fixing typos in constants and in Javadoc? Should make the review of this one a bit easier.
   - Shall we addd validation for TableScanUtil into a separate PR? Comment [here](https://github.com/apache/iceberg/pull/2591#discussion_r649433302).
   - RewriteStrategy
       - Shall we move it to core for now even though it will be public?
       - I am still not sure the current design of `RewriteStrategy` is the best we can come up with. Strategies are Serializable so I assume they can be sent to other nodes in case of distributed planning? However, strategies also define `rewriteFiles` that I think is going to be called on the driver? Plus, right now both the action as well as the strategy interact with the file rewrite coordinator and other things. Ideally, this should be a single entity, in my view. [Here](https://github.com/apache/iceberg/pull/2591#discussion_r634864457) is the relevant 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,552 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build())
+    );
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream)
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroup, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroup.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroup -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {
+      throw e;
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ExecutorService committerService =  Executors.newSingleThreadExecutor(
+            new ThreadFactoryBuilder()
+                .setNameFormat("Committer-Service")
+                .build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())

Review comment:
       Could be simply `groupStream` now?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -93,7 +96,7 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     return Collections.unmodifiableSet(rewrittenDataFiles);
   }
 
-  private Set<DataFile> fetchNewDataFiles(Table table, Set<String> fileSetIDs) {
+  public Set<DataFile> fetchNewDataFiles(Table table, Set<String> fileSetIDs) {

Review comment:
       I think this can now just accept a single file set id.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r649433302



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkFilesScan.java
##########
@@ -62,6 +63,9 @@
     long tableSplitSize = PropertyUtil.propertyAsLong(props, SPLIT_SIZE, SPLIT_SIZE_DEFAULT);
     this.splitSize = Spark3Util.propertyAsLong(options, SparkReadOptions.SPLIT_SIZE, tableSplitSize);
 
+    Preconditions.checkArgument(splitSize > 0,

Review comment:
       I think we better move this validation to `TableScanUtil` and also validate other args such as `lookback`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r634556877



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,440 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+
+    Map<StructLike, Integer> numGroupsPerPartition = fileGroupsByPartition.entrySet().stream()

Review comment:
       Whatever name you think is appropriate.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r635326337



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,505 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = defaultStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * The framework specific strategy to use when no strategy is chosen via an explicit API
+   */
+  protected abstract RewriteStrategy defaultStrategy();
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroupForRewrite> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        List<List<FileScanTask>> fileGroups = toFileGroups(tasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks) {
+    Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+    Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+    return ImmutableList.copyOf(groupedTasks);
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroupForRewrite fileGroupForRewrite,
+                    ConcurrentLinkedQueue<String> completedRewrite,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroupForRewrite.groupID();
+    int filesInGroup = fileGroupForRewrite.numFiles();
+
+    String desc = jobDesc(fileGroupForRewrite, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroupForRewrite.files()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroupForRewrite.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", completedGroupIDs, e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroupForRewrite> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(fileGroupForRewrite -> rewriteFiles(ctx, fileGroupForRewrite, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<FileGroupForRewrite> groupStream) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupRewrites,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() >= groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitOrClean(batch);
+          } catch (Exception e) {
+            batch.forEach(results::remove);
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((fileGroupForRewrite, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", fileGroupForRewrite.info, exception);
+          abortFileGroup(fileGroupForRewrite.groupID()); })

Review comment:
       I didn't like how it looks like
   ```java
   })
   .run
   ```
   But I can switch it back




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r632741217



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();

Review comment:
       I think it would be simpler if you grouped the logic in `files()`, `filterAndGroupFiles()` and closing of resources into one method. You could call it something like `planFileGroups`.
   
   ```
     private Map<StructLike, List<List<FileScanTask>>> planFileGroups(RewriteStrategy strategy) {
       CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
           .filter(filter)
           .ignoreResiduals()
           .planFiles();
   
       try {
         Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
             .collect(Collectors.groupingBy(task -> task.file().partition()));
   
         return filesByPartition.entrySet().stream()
             .collect(Collectors.toMap(Map.Entry::getKey, entry -> toFileGroups(entry.getValue(), strategy)));
   
       } finally {
         try {
           fileScanTasks.close();
         } catch (IOException io) {
           LOG.error("Cannot properly close file iterable while planning for rewrite", io);
         }
       }
     }
   
   ```
   
   I'd also try to keep collectors as simple as possible by putting the logic into a separate method. For example, we can have something like this:
   
   ```
     private List<List<FileScanTask>> toFileGroups(List<FileScanTask> tasks, RewriteStrategy strategy) {
       Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
       Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
       return ImmutableList.copyOf(groupedTasks);
     }
   ```

##########
File path: api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java
##########
@@ -27,7 +27,7 @@
 import org.apache.iceberg.FileScanTask;
 import org.apache.iceberg.Table;
 
-interface RewriteStrategy extends Serializable {
+public interface RewriteStrategy extends Serializable {

Review comment:
       Yeah, I know. That's why I am wondering whether we can move it to core for now.

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java
##########
@@ -50,4 +50,5 @@ private SparkWriteOptions() {
 
   // File scan task set ID that indicates which files must be replaced
   public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id";
+

Review comment:
       No longer needed?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+

Review comment:
       nit: extra line

##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = filterAndGroupFiles(files, strategy);
+
+    try {
+      files.close();
+    } catch (IOException io) {
+      throw new UncheckedIOException("Cannot properly close file iterable while planning for rewrite", io);

Review comment:
       Does this have to fail everything? I'd probably log an error or something.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r657207240



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+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;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(String groupID, List<FileScanTask> filesToRewrite) {
+    manager.stageTasks(table, groupID, filesToRewrite);
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    SparkSession cloneSession = spark.cloneSession();
+    cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+
+    Dataset<Row> scanDF = cloneSession.read().format("iceberg")
+        .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID)
+        .option(SparkReadOptions.SPLIT_SIZE, Long.toString(targetFileSize()))
+        .option(SparkReadOptions.FILE_OPEN_COST, "0")
+        .load(table.name());
+
+    // write the packed data into new files where each split becomes a new file
+
+    try {
+      scanDF.write()

Review comment:
       This is resolved with our "writeMaxFileSize" parameter which changes our write target file size




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664868825



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+

Review comment:
       I think we need to remove redundant logic from this class now. Can be done in a separate PR to simplify the review. We should probably rename it too as it is no longer a commit coordinator.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r665518531



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java
##########
@@ -58,9 +60,19 @@ public void stageTasks(Table table, String setID, List<FileScanTask> tasks) {
     return tasksMap.remove(id);
   }
 
-  private Pair<String, String> toID(Table table, String setID) {
+  public Set<String> fetchSetIDs(Table table) {

Review comment:
       Unfortunately this is iceberg.spark and the test checking this is in iceberg.actions
   
   We have this issue where we have two different hierarchies for packaging actions which we may want to resolve latter but this has to be public for now for me to access it in the tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664869390



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java
##########
@@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set<String> fileSetIDs) {
     table.newRewrite()
         .rewriteFiles(rewrittenDataFiles, newDataFiles)
         .commit();
+

Review comment:
       We can probably align the naming with the task set manager or whatever we call it on the read side.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#issuecomment-866960117


   Note; I have a followup for this to allow RewriteDataFiles to update partitioning of old data files, currently this only works when the partition spec has not changed between old data files and the current status


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664738103



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 com.esotericsoftware.minlog.Log;
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.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
+  );
+
+  private final Table table;
+
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteStrategy strategy;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.strategy = binPackStrategy();
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+  /**
+   * The framework specific {@link BinPackStrategy}
+   */
+  protected abstract BinPackStrategy binPackStrategy();
+
+  @Override
+  public RewriteDataFiles binPack() {
+    this.strategy = binPackStrategy();
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles.Result execute() {
+    validateOptions();
+    strategy = strategy.options(options());
+
+    Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+    Stream<FileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return new Result(Collections.emptyMap());
+    }
+
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream);
+    } else {
+      return doExecute(ctx, groupStream);
+    }
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> planFileGroups() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+
+    try {
+      Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks)
+          .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+      Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap();
+
+      filesByPartition.forEach((partition, tasks) -> {
+        Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks);
+        Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+        List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks);
+        if (fileGroups.size() > 0) {
+          fileGroupsByPartition.put(partition, fileGroups);
+        }
+      });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        fileScanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(RewriteExecutionContext ctx, FileGroup fileGroup,
+                    ConcurrentLinkedQueue<String> rewrittenIDs,
+                    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = fileGroup.groupID();
+    int filesInGroup = fileGroup.numFiles();
+
+    String desc = jobDesc(fileGroup, ctx);
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo("REWRITE-DATA-FILES", desc),
+            () -> strategy.rewriteFiles(groupID, fileGroup.files()));
+
+    rewrittenIDs.offer(groupID);
+    FileGroupRewriteResult fileGroupResult = new FileGroupRewriteResult(addedFiles.size(), filesInGroup);
+
+    results.put(groupID, Pair.of(fileGroup.info(), fileGroupResult));
+  }
+
+  private void commitOrClean(Set<String> rewrittenIDs) {
+    try {
+      commitFileGroups(rewrittenIDs);
+    } catch (CommitStateUnknownException e) {
+      LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.",
+          rewrittenIDs, e);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", rewrittenIDs, e);
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor) Executors.newFixedThreadPool(
+            maxConcurrentFileGroupRewrites,
+            new ThreadFactoryBuilder()
+                .setNameFormat("Rewrite-Service-%d")
+                .build()));
+  }
+
+  private Result doExecute(RewriteExecutionContext ctx, Stream<FileGroup> groupStream) {
+
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<String> rewrittenIDs = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<FileGroup> 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 -> rewriteFiles(ctx, fileGroup, rewrittenIDs, results));
+    } 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, rewrittenIDs.size(), e);
+
+      Tasks.foreach(rewrittenIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitOrClean(ImmutableSet.copyOf(rewrittenIDs));
+    } 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);
+    } catch (Exception e) {

Review comment:
       True we don't need 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] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664730699



##########
File path: core/src/main/java/org/apache/iceberg/util/TableScanUtil.java
##########
@@ -58,6 +58,15 @@ public static boolean hasDeletes(FileScanTask task) {
 
     Function<FileScanTask, Long> weightFunc = file -> Math.max(file.length(), openFileCost);
 
+    Preconditions.checkArgument(splitSize > 0,

Review comment:
       Remove this because we already added it in another pr




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664866161



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java
##########
@@ -58,9 +60,19 @@ public void stageTasks(Table table, String setID, List<FileScanTask> tasks) {
     return tasksMap.remove(id);
   }
 
-  private Pair<String, String> toID(Table table, String setID) {
+  public Set<String> fetchSetIDs(Table table) {

Review comment:
       Is this for testing? If so, can we make it package private?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r632742501



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.math.RoundingMode;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of file groups,
+   * on failure should clean up and rethrow exception
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  private void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(jobStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up all groups which finished being written.", e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitFileGroups(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> jobStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||
+            (!stillRewriting.get() && completedRewriteIds.size() > 0)) {
+
+          Set<String> batch = Sets.newHashSetWithExpectedSize(groupsPerCommit);
+          for (int i = 0; i < groupsPerCommit && !completedRewriteIds.isEmpty(); i++) {
+            batch.add(completedRewriteIds.poll());
+          }
+
+          try {
+            commitFileGroups(batch);
+            completedCommitIds.addAll(batch);
+          } catch (Exception e) {
+            LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e);
+          }
+        }
+      }
+    });
+
+    // Start rewrite tasks
+    Tasks.foreach(jobStream.iterator())
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+          abortFileGroup(info.first().groupID);
+        })
+        .run(infoListPair ->
+            rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewriteIds, results));
+
+    stillRewriting.set(false);
+    committerService.shutdown();
+
+    try {
+      committerService.awaitTermination(10, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      throw new RuntimeException("Cannot complete commit for rewrite, commit service interuptted", e);
+    }
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  @Override
+  public Result execute() {
+    RewriteStrategy strategy =  rewriteStrategy(strategyType).options(this.options());
+    validateOptions(strategy);
+
+    CloseableIterable<FileScanTask> files = files();

Review comment:
       Then it would be just a single line in `execute` that would call `planFileGroups`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r632825320



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();

Review comment:
       no longer needed, part of the more complicated single execute method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r632814887



##########
File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteDataFiles;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+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.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseRewriteDataFilesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS = ImmutableSet.of(
+      MAX_CONCURRENT_FILE_GROUP_ACTIONS,
+      MAX_FILE_GROUP_SIZE_BYTES,
+      PARTIAL_PROGRESS_ENABLED,
+      PARTIAL_PROGRESS_MAX_COMMITS,
+      TARGET_FILE_SIZE_BYTES
+  );
+
+  private final Table table;
+
+  private RewriteDataFiles.Strategy strategyType = Strategy.BINPACK;
+  private Expression filter = Expressions.alwaysTrue();
+  private int maxConcurrentFileGroupActions;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+
+  protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * returns the Spark version specific strategy
+   */
+  protected abstract RewriteStrategy rewriteStrategy(Strategy type);
+
+  private void commitOrClean(Set<String> completedGroupIDs) {
+    try {
+      commitFileGroups(completedGroupIDs);
+    } catch (Exception e) {
+      LOG.error("Cannot commit groups {}, attempting to clean up written files", e);
+      Tasks.foreach(completedGroupIDs)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as
+   * required for this set of file groups
+   * @param completedGroupIDs fileSets to commit
+   */
+  protected abstract void commitFileGroups(Set<String> completedGroupIDs);
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should
+   * not throw any exceptions
+   * @param groupID fileSet to clean
+   */
+  protected abstract void abortFileGroup(String groupID);
+
+
+  @Override
+  public RewriteDataFiles strategy(Strategy type) {
+    strategyType = type;
+    return this;
+  }
+
+  @Override
+  public RewriteDataFiles filter(Expression expression) {
+    filter = Expressions.and(filter, expression);
+    return this;
+  }
+
+  private CloseableIterable<FileScanTask> files() {
+    return table.newScan()
+        .filter(filter)
+        .ignoreResiduals()
+        .planFiles();
+  }
+
+  private Map<StructLike, List<List<FileScanTask>>> filterAndGroupFiles(CloseableIterable<FileScanTask> files,
+      RewriteStrategy strategy) {
+
+    Map<StructLike, List<FileScanTask>> filesByPartition =
+        Streams.stream(files)
+            .collect(Collectors.groupingBy(task -> task.file().partition()));
+
+    return filesByPartition.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> {
+          Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(e.getValue());
+          Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered);
+          return ImmutableList.copyOf(groupedTasks);
+        }));
+  }
+
+  @VisibleForTesting
+  void rewriteFiles(Pair<FileGroupInfo, List<FileScanTask>> infoListPair, int totalGroups,
+      Map<StructLike, Integer> numGroupsPerPartition, RewriteStrategy strategy,
+      ConcurrentLinkedQueue<String> completedRewrite,
+      ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results) {
+
+    String groupID = infoListPair.first().groupID();
+    String desc = jobDesc(infoListPair.first(), totalGroups, infoListPair.second().size(),
+        numGroupsPerPartition.get(infoListPair.first().partition), strategy.name());
+
+    Set<DataFile> addedFiles =
+        withJobGroupInfo(newJobGroupInfo(groupID, desc),
+            () -> strategy.rewriteFiles(groupID, infoListPair.second()));
+
+    completedRewrite.offer(groupID);
+    FileGroupRewriteResult fileGroupResult =
+        new FileGroupRewriteResult(addedFiles.size(), infoListPair.second().size());
+
+    results.put(groupID, Pair.of(infoListPair.first(), fileGroupResult));
+  }
+
+  private Result doExecute(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ConcurrentLinkedQueue<String> completedRewrite = new ConcurrentLinkedQueue<>();
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+
+    Tasks.Builder<Pair<FileGroupInfo, List<FileScanTask>>> rewriteTaskBuilder = Tasks.foreach(groupStream.iterator())
+        .executeWith(rewriteService)
+        .stopOnFailure()
+        .noRetry()
+        .onFailure((info, exception) -> {
+          LOG.error("Failure during rewrite process for group {}", info.first(), exception);
+        });
+
+    try {
+      rewriteTaskBuilder
+          .run(infoListPair ->
+              rewriteFiles(infoListPair, totalGroups, numGroupsPerPartition, strategy, completedRewrite, results));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error("Cannot complete rewrite, partial progress is not enabled and one of the file set groups failed to " +
+          "be rewritten. Cleaning up {} groups which finished being written.", completedRewrite.size(), e);
+      Tasks.foreach(completedRewrite)
+          .suppressFailureWhenFinished()
+          .run(this::abortFileGroup);
+      throw e;
+    }
+
+    commitOrClean(ImmutableSet.copyOf(completedRewrite));
+
+    return new Result(results.values().stream().collect(Collectors.toMap(Pair::first, Pair::second)));
+  }
+
+  private Result doExecutePartialProgress(Stream<Pair<FileGroupInfo, List<FileScanTask>>> groupStream,
+      RewriteStrategy strategy, int totalGroups, Map<StructLike, Integer> numGroupsPerPartition) {
+
+    ExecutorService rewriteService = Executors.newFixedThreadPool(maxConcurrentFileGroupActions,
+        new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build());
+
+    ExecutorService committerService = Executors.newSingleThreadExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Committer-Service").build());
+
+    int groupsPerCommit = partialProgressEnabled ?
+        IntMath.divide(totalGroups, maxCommits, RoundingMode.CEILING) :
+        totalGroups;
+
+    AtomicBoolean stillRewriting = new AtomicBoolean(true);
+    ConcurrentHashMap<String, Pair<FileGroupInfo, FileGroupRewriteResult>> results = new ConcurrentHashMap<>();
+    ConcurrentLinkedQueue<String> completedRewriteIds = new ConcurrentLinkedQueue<>();
+    ConcurrentLinkedQueue<String> completedCommitIds = new ConcurrentLinkedQueue<>();
+
+    // Partial progress commit service
+    committerService.execute(() -> {
+      while (stillRewriting.get() || completedRewriteIds.size() > 0) {
+        Thread.yield();
+        // Either we have a full commit group, or we have completed writing and need to commit what is left over
+        if (completedRewriteIds.size() > groupsPerCommit ||

Review comment:
       Could be >=?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664858990



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitUtil.java
##########
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.io.Closeable;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+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.Lists;
+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.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewriteDataFile Actions from different platforms to handle commits.
+ */
+public class RewriteDataFilesCommitUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitUtil.class);
+
+  private final Table table;
+
+  public RewriteDataFilesCommitUtil(Table table) {
+    this.table = table;
+  }
+
+  /**
+   * 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<RewriteFileGroup> fileGroups) {

Review comment:
       We may be dealing with a lot of files on the driver during compaction. It looks like we are first collecting results in temp sets and then aggregate those sets, coping the data again and again.
   
   In `FileRewriteCoordinator`, I was using `Sets.union` to save some memory. Will that make sense here too? I've seen quite some compaction jobs that were struggling on the driver so I am a bit paranoid :) 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer edited a comment on pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer edited a comment on pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#issuecomment-845645797


   > > If we now want to check if we can remove Delete File A we only have to read files C and D so we actually
   > > made progress.
   > 
   > I think this is the place I am a bit confused. A' and B' don't need delete file A for sure because sequence number of A' and B' is higher. But we don't read C and D to add delete file A to C and D's FileScanTask. It's done by reading the statistics of delete file A and determined by the partition filter. As long as there are files of lower sequence number in that partition, the delete file will be included to that file scan task.
   > 
   > This means that if we can have a counter for each delete file and expose a method `cleanUnreferencedDeleteFiles()` called after `planFileGroups()`, we can naturally get all the files compacted just by running bin packing continuously.
   
   We don't read C and D to check if delete file A is used in C and D's scan task. But we don't know if Delete File A is needed without reading C and D. Delete File A may not touch any rows in files C and D or it may. We can count references, but this ends up being a problem if we don't actually include all files because we have pushed down our filters size filters to the manifest reader itself (future plans) and it only helps us in the luckiest case where we know a delete file has been completely dereferenced. If file C for example is the correct size, and we never need to rewrite it, we never clean up those deletes so we still have to make another sort of action to clean up those files.
   
   That said, the same sweep of delete-files you are describing should probably just be performed on every action which deletes files, and not just during bin pack since that particular method of eliminating files is very cheap but also unlikely to actually pick up any delete files for removal. We talked about this previously as a possible post-merge, post-delete, post-rewrite sort of thing. But again since that kind of clean up never conflicts, we can really do it at any time without a lot of cost to perf.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2591: Spark: RewriteDatafilesAction V2

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r664901065



##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java
##########
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of files to be rewritten by a RewriteAction and the new files which have been
+ * written by the action.
+ */
+public class RewriteFileGroup {
+  private final RewriteDataFiles.FileGroupInfo info;
+  private final List<FileScanTask> fileScanTasks;
+  private final int numInputFiles;
+
+  private Set<DataFile> outputFiles = Collections.emptySet();
+  private int numOutputFiles;
+
+  public RewriteFileGroup(RewriteDataFiles.FileGroupInfo info, List<FileScanTask> fileScanTasks) {
+    this.info = info;
+    this.fileScanTasks = fileScanTasks;
+    this.numInputFiles = fileScanTasks.size();
+  }
+
+  public int numInputFiles() {
+    return numInputFiles;
+  }
+
+  public StructLike partition() {
+    return info.partition();
+  }
+
+  public Integer globalIndex() {
+    return info.globalIndex();
+  }
+
+  public Integer partitionIndex() {
+    return info.partitionIndex();
+  }
+
+  public RewriteDataFiles.FileGroupInfo info() {
+    return info;
+  }
+
+  public List<FileScanTask> fileScans() {
+    return fileScanTasks;
+  }
+
+  public void outputFiles(Set<DataFile> files) {
+    numOutputFiles = files.size();
+    outputFiles = files;
+  }
+
+  public List<DataFile> rewrittenFiles() {

Review comment:
       no just that the addedFiles() was a list. I guess I can make them both sets?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org