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/20 22:01:02 UTC

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

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