aokolnychyi commented on a change in pull request #2591: URL: https://github.com/apache/iceberg/pull/2591#discussion_r664865036
########## File path: spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java ########## @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.math.RoundingMode; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo; +import org.apache.iceberg.actions.BaseRewriteDataFilesResult; +import org.apache.iceberg.actions.BinPackStrategy; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteDataFilesCommitUtil; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.actions.RewriteStrategy; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class BaseRewriteDataFilesSparkAction + extends BaseSnapshotUpdateSparkAction<RewriteDataFiles, RewriteDataFiles.Result> implements RewriteDataFiles { + + private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class); + private static final Set<String> VALID_OPTIONS = ImmutableSet.of( + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_FILE_GROUP_SIZE_BYTES, + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_MAX_COMMITS, + TARGET_FILE_SIZE_BYTES + ); + + private final Table table; + + private Expression filter = Expressions.alwaysTrue(); + private int maxConcurrentFileGroupRewrites; + private int maxCommits; + private boolean partialProgressEnabled; + private RewriteStrategy strategy; + + protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.strategy = binPackStrategy(); + } + + protected Table table() { + return table; + } + + /** + * The framework specific {@link BinPackStrategy} + */ + protected abstract BinPackStrategy binPackStrategy(); + + @Override + public RewriteDataFiles binPack() { + this.strategy = binPackStrategy(); + return this; + } + + @Override + public RewriteDataFiles filter(Expression expression) { + filter = Expressions.and(filter, expression); + return this; + } + + @Override + public RewriteDataFiles.Result execute() { + validateOptions(); + strategy = strategy.options(options()); + + Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = planFileGroups(); + RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition); + Stream<RewriteFileGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition); + + if (ctx.totalGroupCount() == 0) { + LOG.info("Nothing found to rewrite in {}", table.name()); + return new BaseRewriteDataFilesResult(Collections.emptyMap()); + } + + if (partialProgressEnabled) { + return doExecuteWithPartialProgress(ctx, groupStream); + } else { + return doExecute(ctx, groupStream); + } + } + + private Map<StructLike, List<List<FileScanTask>>> planFileGroups() { + CloseableIterable<FileScanTask> fileScanTasks = table.newScan() + .filter(filter) + .ignoreResiduals() + .planFiles(); + + try { + Map<StructLike, List<FileScanTask>> filesByPartition = Streams.stream(fileScanTasks) + .collect(Collectors.groupingBy(task -> task.file().partition())); + + Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition = Maps.newHashMap(); + + filesByPartition.forEach((partition, tasks) -> { + Iterable<FileScanTask> filtered = strategy.selectFilesToRewrite(tasks); + Iterable<List<FileScanTask>> groupedTasks = strategy.planFileGroups(filtered); + List<List<FileScanTask>> fileGroups = ImmutableList.copyOf(groupedTasks); + if (fileGroups.size() > 0) { + fileGroupsByPartition.put(partition, fileGroups); + } + }); + + return fileGroupsByPartition; + } finally { + try { + fileScanTasks.close(); + } catch (IOException io) { + LOG.error("Cannot properly close file iterable while planning for rewrite", io); + } + } + } + + @VisibleForTesting + RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) { + String desc = jobDesc(fileGroup, ctx); + Set<DataFile> addedFiles = withJobGroupInfo( + newJobGroupInfo("REWRITE-DATA-FILES", desc), + () -> strategy.rewriteFiles(fileGroup.fileScans())); + + fileGroup.outputFiles(addedFiles); + LOG.info("Rewrite Files Ready to be Committed - {}", desc); + return fileGroup; + } + + private ExecutorService rewriteService() { + return MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) Executors.newFixedThreadPool( + maxConcurrentFileGroupRewrites, + new ThreadFactoryBuilder() + .setNameFormat("Rewrite-Service-%d") + .build())); + } + + @VisibleForTesting + RewriteDataFilesCommitUtil commitUtil() { + return new RewriteDataFilesCommitUtil(table); + } + + private Result doExecute(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) { + ExecutorService rewriteService = rewriteService(); + RewriteDataFilesCommitUtil commitUtil = commitUtil(); + + ConcurrentLinkedQueue<RewriteFileGroup> rewrittenGroups = Queues.newConcurrentLinkedQueue(); + ConcurrentMap<FileGroupInfo, FileGroupRewriteResult> results = Maps.newConcurrentMap(); + + Tasks.Builder<RewriteFileGroup> rewriteTaskBuilder = Tasks.foreach(groupStream) + .executeWith(rewriteService) + .stopOnFailure() + .noRetry() + .onFailure((fileGroup, exception) -> { + LOG.warn("Failure during rewrite process for group {}", fileGroup.info(), exception); + }); + + try { + rewriteTaskBuilder.run(fileGroup -> { + rewrittenGroups.add(rewriteFiles(ctx, fileGroup)); + }); + } catch (Exception e) { + // At least one rewrite group failed, clean up all completed rewrites + LOG.error("Cannot complete rewrite, {} is not enabled and one of the file set groups failed to " + + "be rewritten. This error occurred during the writing of new files, not during the commit process. This " + + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling " + + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished " + + "being written.", PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED, rewrittenGroups.size(), e); + + Tasks.foreach(rewrittenGroups) + .suppressFailureWhenFinished() + .run(group -> commitUtil.abortFileGroup(group)); + throw e; + } finally { + rewriteService.shutdown(); + } + + try { + commitUtil.commitOrClean(Sets.newHashSet(rewrittenGroups)); + rewrittenGroups.forEach(group -> results.put(group.info(), group.asResult())); + } catch (ValidationException | CommitFailedException e) { + String errorMessage = String.format( + "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that " + + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of " + + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. " + + "Separate smaller rewrite commits can succeed independently while any commits that conflict with " + + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table " + + "history, one for each commit.", + PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage, e); + } + + return new BaseRewriteDataFilesResult(Maps.newHashMap(results)); + } + + private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream<RewriteFileGroup> groupStream) { + ExecutorService rewriteService = rewriteService(); + + // Start Commit Service + int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING); + RewriteDataFilesCommitUtil.CommitService commitService = commitUtil().service(groupsPerCommit); + commitService.start(); + + // Start rewrite tasks + Tasks.foreach(groupStream) + .suppressFailureWhenFinished() + .executeWith(rewriteService) + .noRetry() + .onFailure((fileGroup, exception) -> LOG.error("Failure during rewrite group {}", fileGroup.info(), exception)) + .run(fileGroup -> commitService.offer(rewriteFiles(ctx, fileGroup))); + rewriteService.shutdown(); + + // Stop Commit service + commitService.close(); + List<RewriteFileGroup> commitResults = commitService.results(); + if (commitResults.size() == 0) { + LOG.error("{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " + + "into smaller commits.", PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + } + + return new BaseRewriteDataFilesResult(commitResults.stream() + .collect(Collectors.toMap(RewriteFileGroup::info, RewriteFileGroup::asResult))); + } + + private Stream<RewriteFileGroup> toGroupStream(RewriteExecutionContext ctx, + Map<StructLike, List<List<FileScanTask>>> fileGroupsByPartition) { + + // Todo Add intelligence to the order in which we do rewrites instead of just using partition order + return fileGroupsByPartition.entrySet().stream() + .flatMap( + e -> { Review comment: nit: move `e -> {` to the line above and then reformat the block below? Will reduce the indentation in the closure. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org 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