rdblue commented on a change in pull request #1083:
URL: https://github.com/apache/iceberg/pull/1083#discussion_r434135993



##########
File path: 
spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.spark.source.RowDataRewriter;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RewriteDataFilesAction
+    extends BaseSnapshotUpdateAction<RewriteDataFilesAction, 
RewriteDataFilesActionResult> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteDataFilesAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+  private long rewriteDataFileSizeInBytes;
+  private int splitLookback;
+
+  private PartitionSpec spec = null;
+  private Expression filter;
+
+  RewriteDataFilesAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = new JavaSparkContext(spark.sparkContext());
+    this.table = table;
+    this.spec = table.spec();
+    this.filter = Expressions.alwaysTrue();
+
+    long splitSize = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_SIZE,
+        TableProperties.SPLIT_SIZE_DEFAULT);
+    long targetFileSize = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES,
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT);
+    this.rewriteDataFileSizeInBytes = Math.min(splitSize, targetFileSize);
+
+    this.rewriteDataFileSizeInBytes = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_SIZE,
+        TableProperties.SPLIT_SIZE_DEFAULT);
+    this.splitLookback = PropertyUtil.propertyAsInt(
+        table.properties(),
+        TableProperties.SPLIT_LOOKBACK,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT);
+
+    if (table.io() instanceof HadoopFileIO) {
+      // we need to use Spark's SerializableConfiguration to avoid issues with 
Kryo serialization
+      SerializableConfiguration conf = new 
SerializableConfiguration(((HadoopFileIO) table.io()).conf());
+      this.fileIO = new HadoopFileIO(conf::value);
+    } else {
+      this.fileIO = table.io();
+    }
+    this.encryptionManager = table.encryption();
+  }
+
+  @Override
+  protected RewriteDataFilesAction self() {
+    return this;
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * Pass a PartitionSepc id to specify which PartitionSpec should be used in 
DataFile rewrite
+   *
+   * @param specId PartitionSpec id to rewrite
+   * @return this for method chaining
+   */
+  public RewriteDataFilesAction outputSpecId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  /**
+   * Specify the rewrite data file size in bytes
+   *
+   * @param rewriteDataFileSize size of rewrite data file
+   * @return this for method chaining
+   */
+  public RewriteDataFilesAction rewriteDataFileSizeInBytes(long 
rewriteDataFileSize) {
+    Preconditions.checkArgument(rewriteDataFileSize > 0L, "Invalid rewrite 
data file size in bytes %d",
+        rewriteDataFileSize);
+    this.rewriteDataFileSizeInBytes = rewriteDataFileSize;
+    return this;
+  }
+
+  /**
+   * Specify the split lookback
+   *
+   * @param lookback lookback number to split
+   * @return this for method chaining
+   */
+  public RewriteDataFilesAction splitLookback(int lookback) {
+    Preconditions.checkArgument(lookback > 0L, "Invalid split lookback %d", 
lookback);
+    this.splitLookback = lookback;
+    return this;
+  }
+
+  /**
+   * Pass a row Expression to filter DataFiles to be rewritten. This is 
typically used when only rewriting DatFiles
+   * under some partitions.
+   *
+   * @param expr Expression to filter out DataFiles
+   * @return this for method chaining
+   */
+  public RewriteDataFilesAction filter(Expression expr) {
+    this.filter = Expressions.and(filter, expr);
+    return this;
+  }
+
+  @Override
+  public RewriteDataFilesActionResult execute() {
+    CloseableIterable<FileScanTask> fileScanTasks = table.newScan()
+        .filter(filter)
+        .planFiles();
+
+    Map<StructLikeWrapper, List<FileScanTask>> groupedTasks = 
groupTasksByPartition(fileScanTasks.iterator());
+    // Nothing to rewrite if the table is empty.
+    if (groupedTasks.isEmpty()) {
+      return RewriteDataFilesActionResult.empty();
+    }
+
+    Map<StructLikeWrapper, List<FileScanTask>> filteredGroupedTasks = 
groupedTasks.entrySet().stream()
+        .filter(kv -> kv.getValue().size() > 1)
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    // Nothing to rewrite if there's only one DataFile in each partition.
+    if (filteredGroupedTasks.isEmpty()) {
+      return RewriteDataFilesActionResult.empty();
+    }
+
+    long openFileCost = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_OPEN_FILE_COST,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+
+    // Split and combine tasks under each partition
+    List<CloseableIterable<CombinedScanTask>> groupedCombinedTasks = 
filteredGroupedTasks.values().stream()
+        .map(scanTasks -> {
+          CloseableIterable<FileScanTask> splitTasks = 
TableScanUtil.splitFiles(
+              CloseableIterable.withNoopClose(scanTasks), 
rewriteDataFileSizeInBytes);
+          return TableScanUtil.planTasks(splitTasks, 
rewriteDataFileSizeInBytes, splitLookback, openFileCost);
+        })
+        .collect(Collectors.toList());
+    List<CombinedScanTask> combinedScanTasks = groupedCombinedTasks.stream()
+        .flatMap(Streams::stream)
+        .collect(Collectors.toList());

Review comment:
       Why not combine this with the previous statement?




----------------------------------------------------------------
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