aokolnychyi commented on code in PR #7389: URL: https://github.com/apache/iceberg/pull/7389#discussion_r1179683211
########## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java: ########## @@ -0,0 +1,138 @@ +/* + * 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 static org.apache.spark.sql.functions.col; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.stream.IntStream; +import org.apache.iceberg.DataFilesTable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkTableCache; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter { + + private final SparkSession spark; + private final SparkTableCache tableCache = SparkTableCache.get(); + private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + private final PositionDeletesRewriteCoordinator coordinator = + PositionDeletesRewriteCoordinator.get(); + + SparkPositionDeletesRewriter(SparkSession spark, Table table) { + super(table); + this.spark = spark; + } + + @Override + public String description() { + return "POSITION-DELETES"; + } + + @Override + public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) { + String groupId = UUID.randomUUID().toString(); + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES); + try { + tableCache.add(groupId, deletesTable); + taskSetManager.stageTasks(deletesTable, groupId, group); + + doRewrite(groupId, group); + + return coordinator.fetchNewFiles(deletesTable, groupId); + } finally { + tableCache.remove(groupId); + taskSetManager.removeTasks(deletesTable, groupId); + coordinator.clearRewrite(deletesTable, groupId); + } + } + + protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) { + // All position deletes are of the same partition, because they are in same file group + Preconditions.checkArgument(group.size() > 0, "Empty group"); + Types.StructType partitionType = group.get(0).spec().partitionType(); + StructLike partition = group.get(0).partition(); + + // read the deletes packing them into splits of the required size + Dataset<Row> posDeletes = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) + .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group))) + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .load(groupId); + + // keep only valid position deletes + Dataset<Row> dataFiles = dataFiles(partitionType, partition); + Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path")); + Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi"); + + // write the packed deletes into new files where each split becomes a new file + joined + .sortWithinPartitions("file_path", "pos") + .write() + .format("iceberg") + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize()) + .mode("append") + .save("default.test_table" + ".position_deletes"); + } + + /** Returns entries of {@link DataFilesTable} of specified partition */ + private Dataset<Row> dataFiles(Types.StructType partitionType, StructLike partition) { + List<Types.NestedField> fields = partitionType.fields(); + Optional<Column> condition = + IntStream.range(0, fields.size()) + .mapToObj( + i -> { + Class<?> type = fields.get(i).type().typeId().javaClass(); + Object value = partition.get(i, type); + Column col = col("partition." + fields.get(i).name()); + return col.equalTo(value); + }) + .reduce(Column::and); + if (condition.isPresent()) { + return SparkTableUtil.loadMetadataTable(spark, table(), MetadataTableType.DATA_FILES) Review Comment: question: Do we have to scan the same version of the table? It is OK if someone modifies it in the meantime? Seems like it is not required but we will need to think a bit more. -- 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]
