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: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org