openinx commented on a change in pull request #2216: URL: https://github.com/apache/iceberg/pull/2216#discussion_r588083878
########## 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? + List<Pair<StructLike, CombinedScanTask>> combinedScanTasks = groupedTasks.entrySet().stream() + .map(entry -> { + CloseableIterable<FileScanTask> splitTasks = TableScanUtil.splitFiles( + CloseableIterable.withNoopClose(entry.getValue()), targetSizeInBytes); + return Pair.of(entry.getKey().get(), + TableScanUtil.planTasks(splitTasks, targetSizeInBytes, splitLookback, splitOpenFileCost)); + }) + .flatMap(pair -> StreamSupport.stream(CloseableIterable + .transform(pair.second(), task -> Pair.of(pair.first(), task)).spliterator(), false) + ) + .collect(Collectors.toList()); + + if (!combinedScanTasks.isEmpty()) { + JavaRDD<Pair<StructLike, CombinedScanTask>> taskRDD = sparkContext.parallelize(combinedScanTasks, + combinedScanTasks.size()); + Broadcast<FileIO> io = sparkContext.broadcast(fileIO()); + Broadcast<EncryptionManager> encryption = sparkContext.broadcast(encryptionManager()); + + DeleteRewriter deleteRewriter = new DeleteRewriter(table, caseSensitive, io, encryption); + List<DeleteFile> posDeletes = deleteRewriter.toPosDeletes(taskRDD); Review comment: I'd like to move the RDD chaining out of the [DeleteRewriter](https://github.com/apache/iceberg/pull/2216/files#diff-8735e213fd3c1ca6eef5e7ac82ff5e8cb76addecede2e5cc1912a1dac9ac618fR87-R93) class , so that we could reuse that class for other compute engine's ReplaceDeleteAction. ```java List<DeleteFile> posDeletes = taskRDD.map(deleteRewriter::toPosDeletes) .collect() .stream() .flatMap(Collection::stream) .collect(Collectors.toList()); ``` ########## 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? + List<Pair<StructLike, CombinedScanTask>> combinedScanTasks = groupedTasks.entrySet().stream() + .map(entry -> { + CloseableIterable<FileScanTask> splitTasks = TableScanUtil.splitFiles( + CloseableIterable.withNoopClose(entry.getValue()), targetSizeInBytes); + return Pair.of(entry.getKey().get(), + TableScanUtil.planTasks(splitTasks, targetSizeInBytes, splitLookback, splitOpenFileCost)); + }) + .flatMap(pair -> StreamSupport.stream(CloseableIterable + .transform(pair.second(), task -> Pair.of(pair.first(), task)).spliterator(), false) + ) + .collect(Collectors.toList()); + + if (!combinedScanTasks.isEmpty()) { + JavaRDD<Pair<StructLike, CombinedScanTask>> taskRDD = sparkContext.parallelize(combinedScanTasks, + combinedScanTasks.size()); + Broadcast<FileIO> io = sparkContext.broadcast(fileIO()); + Broadcast<EncryptionManager> encryption = sparkContext.broadcast(encryptionManager()); + + DeleteRewriter deleteRewriter = new DeleteRewriter(table, caseSensitive, io, encryption); + List<DeleteFile> posDeletes = deleteRewriter.toPosDeletes(taskRDD); Review comment: OK, the `DeleteRewriter` is still using few other spark's class such as SparkAppenderFactory. We may need to abstract that part logics, so that we could reuse the rewrite logics between different engines. ########## 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(); Review comment: Nit: I think the `eqDeletes` should better be defined as `HashSet` because different `FileScanTask` will share the same equality delete files (Though we've use the HashSet to deduplicate the same equality delete files in `RewriteFiles` , I still think it's better to do this before calling that API). ########## File path: data/src/main/java/org/apache/iceberg/data/DeleteFilter.java ########## @@ -110,7 +110,44 @@ protected long pos(T record) { return applyEqDeletes(applyPosDeletes(records)); } - private CloseableIterable<T> applyEqDeletes(CloseableIterable<T> records) { + public CloseableIterable<T> matchEqDeletes(CloseableIterable<T> records) { + if (eqDeletes.isEmpty()) { + return records; + } + + Multimap<Set<Integer>, DeleteFile> filesByDeleteIds = Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList); + for (DeleteFile delete : eqDeletes) { + filesByDeleteIds.put(Sets.newHashSet(delete.equalityFieldIds()), delete); + } + + CloseableIterable<T> remainRecords = records; + CloseableIterable<T> matchedRecords = CloseableIterable.empty(); + for (Map.Entry<Set<Integer>, Collection<DeleteFile>> entry : filesByDeleteIds.asMap().entrySet()) { + Set<Integer> ids = entry.getKey(); + Iterable<DeleteFile> deletes = entry.getValue(); + + Schema deleteSchema = TypeUtil.select(requiredSchema, ids); + + // a projection to select and reorder fields of the file schema to match the delete rows + StructProjection projectRow = StructProjection.create(requiredSchema, deleteSchema); + + Iterable<CloseableIterable<Record>> deleteRecords = Iterables.transform(deletes, + delete -> openDeletes(delete, deleteSchema)); + StructLikeSet deleteSet = Deletes.toEqualitySet( + // copy the delete records because they will be held in a set + CloseableIterable.transform(CloseableIterable.concat(deleteRecords), Record::copy), + deleteSchema.asStruct()); + + matchedRecords = CloseableIterable.concat(Lists.newArrayList(matchedRecords, Deletes.match(remainRecords, Review comment: Here I'm concerning it may not worth to take such a high complexity. Let's define the whole data set as `S`, for the first equality field ids `<1,2>`, the deleteSet is `S1`, the second equality field ids `<1,2>`, the deleteSet is `S2`, the third equality delete field ids `<2,5>`, the deleteSet is `S3`. Finally the concat `matchedRecords` will be ` Intersection(S, S1) UNION Intersection(( S - S1 ), S2) UNION Intersection((S - S1- S2), S3)` ( Here `S - S1` means it will return all elements which is in set `S` but not in set `S1` ) though the current code will return the correct converted positional deletions , but it will iterate the big data set `S` three times ? This overhead will be very large... ---------------------------------------------------------------- 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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
