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