simonsssu commented on a change in pull request #1624:
URL: https://github.com/apache/iceberg/pull/1624#discussion_r507423811



##########
File path: 
core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionBase.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.Collection;
+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.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.ListMultimap;
+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.Multimaps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class RewriteDataFilesActionBase<ThisT, R> extends 
BaseSnapshotUpdateAction<ThisT, R> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteDataFilesActionBase.class);
+
+  private final Table table;
+  private final FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+
+  private Expression filter;
+  private long targetSizeInBytes;
+  private int splitLookback;
+  private long splitOpenFileCost;
+  private PartitionSpec spec = null;
+  private boolean caseSensitive = false;
+

Review comment:
       Useless blank line.

##########
File path: 
core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionBase.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.Collection;
+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.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.ListMultimap;
+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.Multimaps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class RewriteDataFilesActionBase<ThisT, R> extends 
BaseSnapshotUpdateAction<ThisT, R> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RewriteDataFilesActionBase.class);
+
+  private final Table table;
+  private final FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+
+  private Expression filter;
+  private long targetSizeInBytes;
+  private int splitLookback;
+  private long splitOpenFileCost;
+  private PartitionSpec spec = null;
+  private boolean caseSensitive = false;
+
+
+  public RewriteDataFilesActionBase(Table table) {
+    this.table = table;
+    fileIO = table.io();
+    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.splitLookback = PropertyUtil.propertyAsInt(
+        table.properties(),
+        TableProperties.SPLIT_LOOKBACK,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT);
+    this.splitOpenFileCost = PropertyUtil.propertyAsLong(
+        table.properties(),
+        TableProperties.SPLIT_OPEN_FILE_COST,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+    this.targetSizeInBytes = Math.min(splitSize, targetFileSize);
+    this.encryptionManager = table.encryption();
+    spec = table.spec();
+  }
+
+
+  /**
+   * Pass a row Expression to filter DataFiles to be rewritten. Note that all 
files that may contain data matching the
+   * filter may be rewritten.
+   *
+   * @param expr Expression to filter out DataFiles
+   * @return this for method chaining
+   */
+
+  public RewriteDataFilesActionBase<ThisT, R> filter(Expression expr) {
+    this.filter = Expressions.and(filter, expr);
+    return this;
+  }
+
+  /**
+   * Pass a PartitionSpec id to specify which PartitionSpec should be used in 
DataFile rewrite
+   *
+   * @param specId PartitionSpec id to rewrite
+   * @return this for method chaining
+   */
+  public RewriteDataFilesActionBase<ThisT, R> outputSpecId(int specId) {
+    Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid 
spec id %d", specId);
+    this.spec = table.specs().get(specId);
+    return this;
+  }
+
+  /**
+   * Specify the number of "bins" considered when trying to pack the next file 
split into a task. Increasing this
+   * usually makes tasks a bit more even by considering more ways to pack file 
regions into a single task with extra
+   * planning cost.
+   * <p>
+   * This configuration can reorder the incoming file regions, to preserve 
order for lower/upper bounds in file
+   * metadata, user can use a lookback of 1.
+   *
+   * @param lookback number of "bins" considered when trying to pack the next 
file split into a task.
+   * @return this for method chaining
+   */
+  public RewriteDataFilesActionBase<ThisT, R> splitLookback(int lookback) {
+    Preconditions.checkArgument(lookback > 0L, "Invalid split lookback %d", 
lookback);
+    this.splitLookback = lookback;
+    return this;
+  }
+
+  /**
+   * Specify the minimum file size to count to pack into one "bin". If the 
read file size is smaller than this specified
+   * threshold, Iceberg will use this value to do count.
+   * <p>
+   * this configuration controls the number of files to compact for each task, 
small value would lead to a high
+   * compaction, the default value is 4MB.
+   *
+   * @param openFileCost minimum file size to count to pack into one "bin".
+   * @return this for method chaining
+   */
+  public RewriteDataFilesActionBase<ThisT, R> splitOpenFileCost(long 
openFileCost) {
+    Preconditions.checkArgument(openFileCost > 0L, "Invalid split openFileCost 
%d", openFileCost);
+    this.splitOpenFileCost = openFileCost;
+    return this;
+  }
+
+  protected void replaceDataFiles(Iterable<DataFile> deletedDataFiles, 
Iterable<DataFile> addedDataFiles) {
+    try {
+      RewriteFiles rewriteFiles = table.newRewrite();
+      rewriteFiles.rewriteFiles(Sets.newHashSet(deletedDataFiles), 
Sets.newHashSet(addedDataFiles));
+      rewriteFiles.commit();
+
+    } catch (Exception e) {
+      Tasks.foreach(Iterables.transform(addedDataFiles, f -> 
f.path().toString()))
+          .noRetry()
+          .suppressFailureWhenFinished()
+          .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", 
location, exc))
+          .run(fileIO::deleteFile);
+
+      throw e;
+    }
+  }
+
+  protected Map<StructLikeWrapper, Collection<FileScanTask>> 
groupTasksByPartition(
+      CloseableIterator<FileScanTask> tasksIter) {
+    ListMultimap<StructLikeWrapper, FileScanTask> tasksGroupedByPartition = 
Multimaps.newListMultimap(
+        Maps.newHashMap(), Lists::newArrayList);
+
+    try {

Review comment:
       can we use try-with-resource to simply the logic of handle exception 
here ? 




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

Reply via email to