You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "szehon-ho (via GitHub)" <gi...@apache.org> on 2023/05/03 19:42:57 UTC

[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

szehon-ho commented on code in PR #7389:
URL: https://github.com/apache/iceberg/pull/7389#discussion_r1184181421


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.Comparator;
+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.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+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.Lists;
+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.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.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);

Review Comment:
   Changed.  It was copied it from RewriteDataFilesSparkAction, I guess we were very excited to commit here :p



-- 
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: issues-unsubscribe@iceberg.apache.org

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