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



##########
File path: 
spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.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.RewriteDataFilesCommitManager;
+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.emptyList());
+    }
+
+    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.setOutputFiles(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
+  RewriteDataFilesCommitManager commitUtil() {

Review comment:
       Shall this be called `commitManager` after the recent rename?

##########
File path: 
core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupRewriteResult.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.actions;
+
+import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewriteDataFiles.FileGroupRewriteResult;
+
+public class BaseRewriteDataFilesFileGroupRewriteResult implements 
FileGroupRewriteResult {

Review comment:
       nit: just `BaseFileGroupRewriteResult`?

##########
File path: 
spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.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.RewriteDataFilesCommitManager;
+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.emptyList());
+    }
+
+    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.setOutputFiles(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
+  RewriteDataFilesCommitManager commitUtil() {

Review comment:
       Variables that refer to it should be renamed too.

##########
File path: 
spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.BinPackStrategy;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+
+public class Spark3BinPackStrategy extends BinPackStrategy {
+  private final Table table;
+  private final SparkSession spark;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = 
FileRewriteCoordinator.get();
+
+  public Spark3BinPackStrategy(Table table, SparkSession spark) {
+    this.table = table;
+    this.spark = spark;
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public Set<DataFile> rewriteFiles(List<FileScanTask> filesToRewrite) {
+    Set<DataFile> results;
+
+    String groupID = UUID.randomUUID().toString();
+    try {
+      manager.stageTasks(table, groupID, filesToRewrite);
+
+      // Disable Adaptive Query Execution as this may change the output 
partitioning of our write
+      SparkSession cloneSession = spark.cloneSession();

Review comment:
       I did look into the code a little bit. Seems the current assumption is 
that the state will be GCed once a session becomes unreachable?

##########
File path: core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of files to be rewritten by a 
RewriteAction and the new files which have been
+ * written by the action.
+ */
+public class RewriteFileGroup {

Review comment:
       It looks like we use this class in debug messages but don't provide a 
reasonable `toString` implementation.




-- 
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: [email protected]

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