aokolnychyi commented on a change in pull request #2591:
URL: https://github.com/apache/iceberg/pull/2591#discussion_r634043801



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

Review comment:
       This method uses `infoListPair.first()` and `infoListPair.second()` 
quite a bit and I keep going back to see what each of those mean. Also, 
`infoListPair` isn't very descriptive and `Pair<FileGroupInfo, 
List<FileScanTask>>` is kind of bulky.
   
   While looking for a slightly more descriptive name, I saw it is also called 
a group in other places. What about creating a private helper class instead of 
using `Pair`? Also, what about creating temp vars that refer to the info and 
files? That should shorten other lines.
   
   ```
     private static class Group {
       private final FileGroupInfo groupInfo;
       private final List<FileScanTask> files;
   
       Group(FileGroupInfo groupInfo, List<FileScanTask> files) {
         this.groupInfo = groupInfo;
         this.files = files;
       }
   
       public FileGroupInfo info() {
         return groupInfo;
       }
   
       public List<FileScanTask> files() {
         return files;
       }
     }
   ```
   
   Then we can have `toGroupStream` that would construct a stream of groups 
instead of having `toJobStream` that returns `Stream<Pair<FileGroupInfo, 
List<FileScanTask>>>`.
   
   Then `rewriteFiles` can accept an instance of `Group`.
   




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to