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



##########
File path: 
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static 
org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static 
org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, 
RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, 
SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf 
writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] 
requiredOrdering) {
+    this.sparkContext = 
JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, 
Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) 
{
+      // broadcast the table metadata as the writer factory will be sent to 
executors
+      Broadcast<Table> tableBroadcast = 
sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        
referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty 
relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend 
on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: 
%s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), 
conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no 
validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {

Review comment:
       I think it would be better to make this `static` and pass in `scan`. 
That's cleaner because right now this requires being wrapped in a check that 
`scan` is not null. Instead of having methods that require certain states for 
fields, I prefer to have static methods that make their args explicit. That is, 
if you pass in `scan` you see that it should be non-null.




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