You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/03/05 06:44:31 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #2216: Spark: support replace equality deletes to position deletes

openinx commented on a change in pull request #2216:
URL: https://github.com/apache/iceberg/pull/2216#discussion_r588067080



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ReplaceDeleteAction.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+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.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.spark.SparkUtil;
+import org.apache.iceberg.spark.source.DeleteRewriter;
+import org.apache.iceberg.util.Pair;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplaceDeleteAction extends
+    BaseSnapshotUpdateAction<ReplaceDeleteAction, DeleteRewriteActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ReplaceDeleteAction.class);
+  private final Table table;
+  private final JavaSparkContext sparkContext;
+  private FileIO fileIO;
+  private final EncryptionManager encryptionManager;
+  private final boolean caseSensitive;
+  private final PartitionSpec spec;
+  private final long targetSizeInBytes;
+  private final int splitLookback;
+  private final long splitOpenFileCost;
+
+  public ReplaceDeleteAction(SparkSession spark, Table table) {
+    this.table = table;
+    this.sparkContext = new JavaSparkContext(spark.sparkContext());
+    this.fileIO = fileIO();
+    this.encryptionManager = table.encryption();
+    this.caseSensitive = false;
+    this.spec = table.spec();
+
+    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.targetSizeInBytes = Math.min(splitSize, targetFileSize);
+
+    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);
+  }
+
+  protected FileIO fileIO() {
+    if (this.fileIO == null) {
+      this.fileIO = SparkUtil.serializableFileIO(table());
+    }
+    return this.fileIO;
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  @Override
+  public DeleteRewriteActionResult execute() {
+    CloseableIterable<FileScanTask> fileScanTasks = null;
+    try {
+      fileScanTasks = table.newScan()
+          .caseSensitive(caseSensitive)
+          .ignoreResiduals()
+          .planFiles();
+    } finally {
+      try {
+        if (fileScanTasks != null) {
+          fileScanTasks.close();
+        }
+      } catch (IOException ioe) {
+        LOG.warn("Failed to close task iterable", ioe);
+      }
+    }
+
+    CloseableIterable<FileScanTask> tasksWithEqDelete = CloseableIterable.filter(fileScanTasks, scan ->
+        scan.deletes().stream().anyMatch(delete -> delete.content().equals(FileContent.EQUALITY_DELETES))
+    );
+
+    List<DeleteFile> eqDeletes = Lists.newArrayList();
+    tasksWithEqDelete.forEach(task -> {
+      eqDeletes.addAll(task.deletes().stream()
+          .filter(deleteFile -> deleteFile.content().equals(FileContent.EQUALITY_DELETES))
+          .collect(Collectors.toList()));
+    });
+
+    Map<StructLikeWrapper, Collection<FileScanTask>> groupedTasks = groupTasksByPartition(tasksWithEqDelete.iterator());
+
+    // Split and combine tasks under each partition
+    // TODO: can we split task?

Review comment:
       Yes,  we can split the task based on `DataFile`(s) here.  But that introduce another issue here,  the current balance policy ( for splitting tasks ) only consider the `DataFile` ,  the idea way should be considering both insert file size and delete file size.  I think there should be another separate issues to address this.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org