diff --git a/core/src/main/java/org/apache/iceberg/actions/FileRewriteExecutor.java b/core/src/main/java/org/apache/iceberg/actions/FileRewriteExecutor.java new file mode 100644 index 000000000000..5d589c6931c5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/FileRewriteExecutor.java @@ -0,0 +1,88 @@ +/* + * 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.util.Map; +import java.util.Set; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; + +/** + * A class for rewriting content file groups ({@link FileRewriteGroup}). The lifecycle for the + * executor looks like the following: + * + * + * + * A single executor could be used to rewrite multiple groups for the same plan. + * + * @param the Java type of the plan info + * @param the Java type of the tasks to read content files + * @param the Java type of the content files + * @param the Java type of the planned groups + * @param

the Java type of the plan to execute + */ +public interface FileRewriteExecutor< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends FileRewriteGroup, + P extends FileRewritePlan> { + + /** Returns a description for this rewriter. */ + default String description() { + return getClass().getName(); + } + + /** + * Returns a set of supported options for this rewriter. Only options specified in this list will + * be accepted at runtime. Any other options will be rejected. + */ + Set validOptions(); + + /** + * Initializes this rewriter using provided options. + * + * @param options options to initialize this rewriter + */ + void init(Map options); + + /** + * Initializes the rewriter using the information generated during planning. + * + * @param plan containing the configuration data + */ + void initPlan(P plan); + + /** + * Rewrite a group of files represented by the given list of scan tasks. + * + *

The implementation is supposed to be engine-specific (e.g. Spark, Flink, Trino). + * + * @param group of scan tasks for files to be rewritten together + * @return a set of newly written files + */ + Set rewrite(G group); +} diff --git a/core/src/main/java/org/apache/iceberg/actions/FileRewriteGroup.java b/core/src/main/java/org/apache/iceberg/actions/FileRewriteGroup.java new file mode 100644 index 000000000000..08f6e050b163 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/FileRewriteGroup.java @@ -0,0 +1,93 @@ +/* + * 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.util.Comparator; +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.RewriteJobOrder; + +/** + * Container class representing a set of files to be rewritten by a {@link FileRewriteExecutor}. + * + * @param the Java type of the plan info + * @param the Java type of the tasks to read content files + * @param the Java type of the content files + */ +public abstract class FileRewriteGroup, F extends ContentFile> { + private final I info; + private final List fileScanTasks; + private final long splitSize; + private final int expectedOutputFiles; + + FileRewriteGroup(I info, List fileScanTasks, long splitSize, int expectedOutputFiles) { + this.info = info; + this.fileScanTasks = fileScanTasks; + this.splitSize = splitSize; + this.expectedOutputFiles = expectedOutputFiles; + } + + /** Identifiers and partition information about the group. */ + public I info() { + return info; + } + + /** Input of the group. {@link ContentScanTask}s to read. */ + public List fileScans() { + return fileScanTasks; + } + + /** Expected split size for the output files. */ + public long splitSize() { + return splitSize; + } + + /** Expected number of the output files. */ + public int expectedOutputFiles() { + return expectedOutputFiles; + } + + /** Accumulated size for the input files. */ + public long sizeInBytes() { + return fileScanTasks.stream().mapToLong(T::length).sum(); + } + + /** Number of the input files. */ + public int numInputFiles() { + return fileScanTasks.size(); + } + + /** Comparator to order the FileRewriteGroups based on a provided {@link RewriteJobOrder}. */ + public static , F extends ContentFile> + Comparator> taskComparator(RewriteJobOrder rewriteJobOrder) { + switch (rewriteJobOrder) { + case BYTES_ASC: + return Comparator.comparing(FileRewriteGroup::sizeInBytes); + case BYTES_DESC: + return Comparator.comparing(FileRewriteGroup::sizeInBytes, Comparator.reverseOrder()); + case FILES_ASC: + return Comparator.comparing(FileRewriteGroup::numInputFiles); + case FILES_DESC: + return Comparator.comparing(FileRewriteGroup::numInputFiles, Comparator.reverseOrder()); + default: + return (unused, unused2) -> 0; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/FileRewritePlan.java b/core/src/main/java/org/apache/iceberg/actions/FileRewritePlan.java new file mode 100644 index 000000000000..f313fd1b070d --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/FileRewritePlan.java @@ -0,0 +1,83 @@ +/* + * 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.util.Map; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.StructLike; + +/** + * Result of the file rewrite planning as generated by the {@link FileRewritePlanner#plan()}. + * + *

The plan contains the stream of the planned groups and statistics about the number of the + * generated groups, like the total number of the groups and the groups per partition. The plan also + * contains some calculated values required by the {@link FileRewriteExecutor}s where the values are + * based on the input data and the planning parameters. + * + *

Groups in a plan could be processed independently. For example, in Spark this means that each + * group would be rewritten in its own Spark job. + * + * @param the Java type of the plan info + * @param the Java type of the tasks to read content files + * @param the Java type of the content files + * @param the Java type of the planned groups + */ +public abstract class FileRewritePlan< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends FileRewriteGroup> { + private final Stream groups; + private final int totalGroupCount; + private final Map groupsInPartition; + private final long writeMaxFileSize; + + protected FileRewritePlan( + Stream groups, + int totalGroupCount, + Map groupsInPartition, + long writeMaxFileSize) { + this.groups = groups; + this.totalGroupCount = totalGroupCount; + this.groupsInPartition = groupsInPartition; + this.writeMaxFileSize = writeMaxFileSize; + } + + /** The stream of the generated {@link FileRewriteGroup}s. */ + public Stream groups() { + return groups; + } + + /** The number of the generated groups in the given partition. */ + public int groupsInPartition(StructLike partition) { + return groupsInPartition.get(partition); + } + + /** The total number of the groups generated by this plan. */ + public int totalGroupCount() { + return totalGroupCount; + } + + /** Calculated maximum file size based on the planner target file size configuration */ + public long writeMaxFileSize() { + return writeMaxFileSize; + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/FileRewritePlanner.java b/core/src/main/java/org/apache/iceberg/actions/FileRewritePlanner.java new file mode 100644 index 000000000000..0d242bbe6bb0 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/FileRewritePlanner.java @@ -0,0 +1,76 @@ +/* + * 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.util.Map; +import java.util.Set; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; + +/** + * A class for planning content file rewrites. + * + *

The entire rewrite operation is broken down into pieces. The grouping is based on partitioning + * and the planning could create multiple groups within a partition. As a result {@link + * FileRewritePlan} is generated which contains the data need by the {@link FileRewriteExecutor}s + * which execute the actual file rewrite. + * + *

The lifecycle of the planner is: + * + *

    + *
  • {@link #init(Map)} initializes the planner with the configuration parameters + *
  • {@link #plan()} generates the plan for the given configuration + *
+ * + * @param the Java type of the plan info + * @param the Java type of the tasks to read content files + * @param the Java type of the content files + * @param the Java type of the planned groups + */ +public interface FileRewritePlanner< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends FileRewriteGroup> { + + /** Returns a description for this rewriter. */ + default String description() { + return getClass().getName(); + } + + /** + * Returns a set of supported options for this rewriter. Only options specified in this list will + * be accepted at runtime. Any other options will be rejected. + */ + Set validOptions(); + + /** + * Initializes this rewriter using provided options. + * + * @param options options to initialize this rewriter + */ + void init(Map options); + + /** + * Generates the plan for rewrite. + * + * @return the generated plan which could be executed during the compaction + */ + FileRewritePlan plan(); +} diff --git a/core/src/main/java/org/apache/iceberg/actions/FileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/FileRewriter.java index 7c6b4e8d7ef5..f014aea0c034 100644 --- a/core/src/main/java/org/apache/iceberg/actions/FileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/FileRewriter.java @@ -34,7 +34,10 @@ * * @param the Java type of tasks to read content files * @param the Java type of content files + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link FileRewritePlanner} and {@link + * FileRewriteExecutor}. */ +@Deprecated public interface FileRewriter, F extends ContentFile> { /** Returns a description for this rewriter. */ diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java index dfc9842780f5..996e7b0f8ba2 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java @@ -31,26 +31,26 @@ import org.apache.iceberg.util.DataFileSet; /** - * Container class representing a set of files to be rewritten by a RewriteAction and the new files - * which have been written by the action. + * Container class representing a set of data files to be rewritten by a RewriteAction and the new + * files which have been written by the action. */ -public class RewriteFileGroup { - private final FileGroupInfo info; - private final List fileScanTasks; - +public class RewriteFileGroup extends FileRewriteGroup { private DataFileSet addedFiles = DataFileSet.create(); + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. + */ + @Deprecated public RewriteFileGroup(FileGroupInfo info, List fileScanTasks) { - this.info = info; - this.fileScanTasks = fileScanTasks; - } - - public FileGroupInfo info() { - return info; + this(info, fileScanTasks, 0L, 0); } - public List fileScans() { - return fileScanTasks; + public RewriteFileGroup( + FileGroupInfo info, + List fileScanTasks, + long splitSize, + int expectedOutputFiles) { + super(info, fileScanTasks, splitSize, expectedOutputFiles); } public void setOutputFiles(Set files) { @@ -70,9 +70,9 @@ public Set addedFiles() { public RewriteDataFiles.FileGroupRewriteResult asResult() { Preconditions.checkState(addedFiles != null, "Cannot get result, Group was never rewritten"); return ImmutableRewriteDataFiles.FileGroupRewriteResult.builder() - .info(info) + .info(info()) .addedDataFilesCount(addedFiles.size()) - .rewrittenDataFilesCount(fileScanTasks.size()) + .rewrittenDataFilesCount(fileScans().size()) .rewrittenBytesCount(sizeInBytes()) .build(); } @@ -80,8 +80,8 @@ public RewriteDataFiles.FileGroupRewriteResult asResult() { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("info", info) - .add("numRewrittenFiles", fileScanTasks.size()) + .add("info", info()) + .add("numRewrittenFiles", fileScans().size()) .add( "numAddedFiles", addedFiles == null ? "Rewrite Incomplete" : Integer.toString(addedFiles.size())) @@ -89,14 +89,19 @@ public String toString() { .toString(); } - public long sizeInBytes() { - return fileScanTasks.stream().mapToLong(FileScanTask::length).sum(); - } - + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. Use {@link #numInputFiles()} instead. + */ + @Deprecated public int numFiles() { - return fileScanTasks.size(); + return fileScans().size(); } + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. Use {@link + * FileRewriteGroup#taskComparator(RewriteJobOrder)} instead. + */ + @Deprecated public static Comparator comparator(RewriteJobOrder rewriteJobOrder) { switch (rewriteJobOrder) { case BYTES_ASC: diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroupPlanner.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroupPlanner.java new file mode 100644 index 000000000000..2ba632fcf061 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroupPlanner.java @@ -0,0 +1,280 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.RewriteJobOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.StructLikeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Groups specified files in the {@link Table} by {@link RewriteFileGroup}s. These will be grouped + * by partitions. Extends {@link SizeBasedFileRewritePlanner} with delete file threshold and job + * {@link RewriteDataFiles#REWRITE_JOB_ORDER} handling. + */ +public class RewriteFileGroupPlanner + extends SizeBasedFileRewritePlanner { + /** + * The minimum number of deletes that needs to be associated with a data file for it to be + * considered for rewriting. If a data file has this number of deletes or more, it will be + * rewritten regardless of its file size determined by {@link #MIN_FILE_SIZE_BYTES} and {@link + * #MAX_FILE_SIZE_BYTES}. If a file group contains a file that satisfies this condition, the file + * group will be rewritten regardless of the number of files in the file group determined by + * {@link #MIN_INPUT_FILES}. + * + *

Defaults to Integer.MAX_VALUE, which means this feature is not enabled by default. + */ + public static final String DELETE_FILE_THRESHOLD = "delete-file-threshold"; + + public static final int DELETE_FILE_THRESHOLD_DEFAULT = Integer.MAX_VALUE; + + private static final Logger LOG = LoggerFactory.getLogger(RewriteFileGroupPlanner.class); + + private final Expression filter; + private final Long snapshotId; + private final boolean caseSensitive; + + private int deleteFileThreshold; + private RewriteJobOrder rewriteJobOrder; + + public RewriteFileGroupPlanner(Table table) { + this(table, Expressions.alwaysTrue()); + } + + public RewriteFileGroupPlanner(Table table, Expression filter) { + this( + table, + filter, + table.currentSnapshot() != null ? table.currentSnapshot().snapshotId() : null, + false); + } + + /** + * Creates the planner for the given table. + * + * @param table to plan for + * @param filter used to remove files from the plan + * @param snapshotId used as a basis for planning - should be used as starting snapshot id at + * commit time when replacing the files + * @param caseSensitive property used for scanning + */ + public RewriteFileGroupPlanner( + Table table, Expression filter, Long snapshotId, boolean caseSensitive) { + super(table); + this.filter = filter; + this.snapshotId = snapshotId; + this.caseSensitive = caseSensitive; + } + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(DELETE_FILE_THRESHOLD) + .add(RewriteDataFiles.REWRITE_JOB_ORDER) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.deleteFileThreshold = deleteFileThreshold(options); + this.rewriteJobOrder = + RewriteJobOrder.fromName( + PropertyUtil.propertyAsString( + options, + RewriteDataFiles.REWRITE_JOB_ORDER, + RewriteDataFiles.REWRITE_JOB_ORDER_DEFAULT)); + } + + @Override + protected Iterable filterFiles(Iterable tasks) { + return Iterables.filter(tasks, task -> wronglySized(task) || tooManyDeletes(task)); + } + + @Override + protected Iterable> filterFileGroups(List> groups) { + return Iterables.filter(groups, this::shouldRewrite); + } + + @Override + protected long defaultTargetFileSize() { + return PropertyUtil.propertyAsLong( + table().properties(), + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + } + + @Override + public RewriteFilePlan plan() { + StructLikeMap>> plan = planFileGroups(); + RewriteExecutionContext ctx = new RewriteExecutionContext(); + Stream groups = + plan.entrySet().stream() + .filter(e -> !e.getValue().isEmpty()) + .flatMap( + e -> { + StructLike partition = e.getKey(); + List> scanGroups = e.getValue(); + return scanGroups.stream() + .map( + tasks -> { + long inputSize = inputSize(tasks); + return newRewriteGroup( + ctx, + partition, + tasks, + splitSize(inputSize), + numOutputFiles(inputSize)); + }); + }) + .sorted(FileRewriteGroup.taskComparator(rewriteJobOrder)); + Map groupsInPartition = plan.transformValues(List::size); + int totalGroupCount = groupsInPartition.values().stream().reduce(Integer::sum).orElse(0); + return new RewriteFilePlan( + groups, totalGroupCount, groupsInPartition, writeMaxFileSize(), outputSpecId()); + } + + private CloseableIterable tasks() { + TableScan scan = + table().newScan().filter(filter).caseSensitive(caseSensitive).ignoreResiduals(); + + if (snapshotId != null) { + scan = scan.useSnapshot(snapshotId); + } + + return scan.planFiles(); + } + + private int deleteFileThreshold(Map options) { + int value = + PropertyUtil.propertyAsInt(options, DELETE_FILE_THRESHOLD, DELETE_FILE_THRESHOLD_DEFAULT); + Preconditions.checkArgument( + value >= 0, "'%s' is set to %s but must be >= 0", DELETE_FILE_THRESHOLD, value); + return value; + } + + private boolean tooManyDeletes(FileScanTask task) { + return task.deletes() != null && task.deletes().size() >= deleteFileThreshold; + } + + private boolean shouldRewrite(List group) { + return enoughInputFiles(group) + || enoughContent(group) + || tooMuchContent(group) + || anyTaskHasTooManyDeletes(group); + } + + private boolean anyTaskHasTooManyDeletes(List group) { + return group.stream().anyMatch(this::tooManyDeletes); + } + + private StructLikeMap>> planFileGroups() { + CloseableIterable fileScanTasks = tasks(); + + try { + Types.StructType partitionType = table().spec().partitionType(); + StructLikeMap> filesByPartition = + groupByPartition(table(), partitionType, fileScanTasks); + return filesByPartition.transformValues(tasks -> ImmutableList.copyOf(planFileGroups(tasks))); + } finally { + try { + fileScanTasks.close(); + } catch (IOException io) { + LOG.error("Cannot properly close file iterable while planning for rewrite", io); + } + } + } + + private StructLikeMap> groupByPartition( + Table table, Types.StructType partitionType, Iterable tasks) { + StructLikeMap> filesByPartition = StructLikeMap.create(partitionType); + StructLike emptyStruct = GenericRecord.create(partitionType); + + for (FileScanTask task : tasks) { + // If a task uses an incompatible partition spec the data inside could contain values + // which belong to multiple partitions in the current spec. Treating all such files as + // un-partitioned and grouping them together helps to minimize new files made. + StructLike taskPartition = + task.file().specId() == table.spec().specId() ? task.file().partition() : emptyStruct; + + filesByPartition.computeIfAbsent(taskPartition, unused -> Lists.newArrayList()).add(task); + } + + return filesByPartition; + } + + private RewriteFileGroup newRewriteGroup( + RewriteExecutionContext ctx, + StructLike partition, + List tasks, + long splitSize, + int numOutputSize) { + FileGroupInfo info = + ImmutableRewriteDataFiles.FileGroupInfo.builder() + .globalIndex(ctx.currentGlobalIndex()) + .partitionIndex(ctx.currentPartitionIndex(partition)) + .partition(partition) + .build(); + return new RewriteFileGroup(info, Lists.newArrayList(tasks), splitSize, numOutputSize); + } + + private static class RewriteExecutionContext { + private final Map partitionIndexMap; + private final AtomicInteger groupIndex; + + private RewriteExecutionContext() { + this.partitionIndexMap = Maps.newConcurrentMap(); + this.groupIndex = new AtomicInteger(1); + } + + private int currentGlobalIndex() { + return groupIndex.getAndIncrement(); + } + + private int currentPartitionIndex(StructLike partition) { + return partitionIndexMap.merge(partition, 1, Integer::sum); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFilePlan.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFilePlan.java new file mode 100644 index 000000000000..b6d9a35ebd9f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFilePlan.java @@ -0,0 +1,47 @@ +/* + * 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.util.Map; +import java.util.stream.Stream; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.StructLike; + +/** Result of the data file rewrite planning. */ +public class RewriteFilePlan + extends FileRewritePlan< + RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, RewriteFileGroup> { + private final int outputSpecId; + + public RewriteFilePlan( + Stream groups, + int totalGroupCount, + Map groupsInPartition, + long writeMaxFileSize, + int outputSpecId) { + super(groups, totalGroupCount, groupsInPartition, writeMaxFileSize); + this.outputSpecId = outputSpecId; + } + + /** Partition specification id for the target files */ + public int outputSpecId() { + return outputSpecId; + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletePlan.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletePlan.java new file mode 100644 index 000000000000..15ee241ad99e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletePlan.java @@ -0,0 +1,41 @@ +/* + * 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.util.Map; +import java.util.stream.Stream; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.StructLike; + +/** Result of the positional delete file rewrite planning. */ +public class RewritePositionDeletePlan + extends FileRewritePlan< + RewritePositionDeleteFiles.FileGroupInfo, + PositionDeletesScanTask, + DeleteFile, + RewritePositionDeletesGroup> { + public RewritePositionDeletePlan( + Stream groups, + int totalGroupCount, + Map groupsInPartition, + long writeMaxFileSize) { + super(groups, totalGroupCount, groupsInPartition, writeMaxFileSize); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java index d1c688417a64..c7b1f9ddaf51 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java @@ -35,27 +35,37 @@ * Container class representing a set of position delete files to be rewritten by a {@link * RewritePositionDeleteFiles} and the new files which have been written by the action. */ -public class RewritePositionDeletesGroup { - private final FileGroupInfo info; - private final List tasks; +public class RewritePositionDeletesGroup + extends FileRewriteGroup { private final long maxRewrittenDataSequenceNumber; private DeleteFileSet addedDeleteFiles = DeleteFileSet.create(); + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. + */ + @Deprecated public RewritePositionDeletesGroup(FileGroupInfo info, List tasks) { + this(info, tasks, 0L, 0); + } + + public RewritePositionDeletesGroup( + FileGroupInfo info, + List tasks, + long splitSize, + int expectedOutputFiles) { + super(info, tasks, splitSize, expectedOutputFiles); Preconditions.checkArgument(!tasks.isEmpty(), "Tasks must not be empty"); - this.info = info; - this.tasks = tasks; this.maxRewrittenDataSequenceNumber = tasks.stream().mapToLong(t -> t.file().dataSequenceNumber()).max().getAsLong(); } - public FileGroupInfo info() { - return info; - } - + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. Use {@link #fileScans()} instead. + */ + @Deprecated public List tasks() { - return tasks; + return fileScans(); } public void setOutputFiles(Set files) { @@ -67,7 +77,7 @@ public long maxRewrittenDataSequenceNumber() { } public Set rewrittenDeleteFiles() { - return tasks().stream() + return fileScans().stream() .map(PositionDeletesScanTask::file) .collect(Collectors.toCollection(DeleteFileSet::create)); } @@ -81,9 +91,9 @@ public FileGroupRewriteResult asResult() { addedDeleteFiles != null, "Cannot get result, Group was never rewritten"); return ImmutableRewritePositionDeleteFiles.FileGroupRewriteResult.builder() - .info(info) + .info(info()) .addedDeleteFilesCount(addedDeleteFiles.size()) - .rewrittenDeleteFilesCount(tasks.size()) + .rewrittenDeleteFilesCount(fileScans().size()) .rewrittenBytesCount(rewrittenBytes()) .addedBytesCount(addedBytes()) .build(); @@ -92,8 +102,8 @@ public FileGroupRewriteResult asResult() { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("info", info) - .add("numRewrittenPositionDeleteFiles", tasks.size()) + .add("info", info()) + .add("numRewrittenPositionDeleteFiles", fileScans().size()) .add( "numAddedPositionDeleteFiles", addedDeleteFiles == null @@ -105,17 +115,26 @@ public String toString() { } public long rewrittenBytes() { - return tasks.stream().mapToLong(PositionDeletesScanTask::length).sum(); + return fileScans().stream().mapToLong(PositionDeletesScanTask::length).sum(); } public long addedBytes() { return addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); } + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. Use {@link #numInputFiles()} instead. + */ + @Deprecated public int numRewrittenDeleteFiles() { - return tasks.size(); + return fileScans().size(); } + /** + * @deprecated since 1.8.0, will be removed in 1.9.0. Use {@link + * FileRewriteGroup#taskComparator(RewriteJobOrder)} instead. + */ + @Deprecated public static Comparator comparator(RewriteJobOrder order) { switch (order) { case BYTES_ASC: diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroupPlanner.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroupPlanner.java new file mode 100644 index 000000000000..14bd3cabefcc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroupPlanner.java @@ -0,0 +1,241 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +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.PositionDeletesTable; +import org.apache.iceberg.RewriteJobOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupInfo; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.StructLikeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Groups specified files in the {@link Table} by {@link RewriteFileGroup}s. These will be grouped + * by partitions. Extends the {@link SizeBasedFileRewritePlanner} with {@link + * RewritePositionDeleteFiles#REWRITE_JOB_ORDER} handling. + */ +public class RewritePositionDeletesGroupPlanner + extends SizeBasedFileRewritePlanner< + FileGroupInfo, PositionDeletesScanTask, DeleteFile, RewritePositionDeletesGroup> { + private static final Logger LOG = + LoggerFactory.getLogger(RewritePositionDeletesGroupPlanner.class); + + private final Expression filter; + private final boolean caseSensitive; + private RewriteJobOrder rewriteJobOrder; + + public RewritePositionDeletesGroupPlanner(Table table) { + this(table, Expressions.alwaysTrue(), false); + } + + /** + * Creates the planner for the given table. + * + * @param table to plan for + * @param filter used to remove files from the plan + * @param caseSensitive property used for scanning + */ + public RewritePositionDeletesGroupPlanner(Table table, Expression filter, boolean caseSensitive) { + super(table); + this.caseSensitive = caseSensitive; + this.filter = filter; + } + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(RewritePositionDeleteFiles.REWRITE_JOB_ORDER) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.rewriteJobOrder = + RewriteJobOrder.fromName( + PropertyUtil.propertyAsString( + options, + RewritePositionDeleteFiles.REWRITE_JOB_ORDER, + RewritePositionDeleteFiles.REWRITE_JOB_ORDER_DEFAULT)); + } + + @Override + public RewritePositionDeletePlan plan() { + StructLikeMap>> plan = planFileGroups(); + RewriteExecutionContext ctx = new RewriteExecutionContext(); + Stream groups = + plan.entrySet().stream() + .filter(e -> !e.getValue().isEmpty()) + .flatMap( + e -> { + StructLike partition = e.getKey(); + List> scanGroups = e.getValue(); + return scanGroups.stream() + .map( + tasks -> { + long inputSize = inputSize(tasks); + return newRewriteGroup( + ctx, + partition, + tasks, + splitSize(inputSize), + numOutputFiles(inputSize)); + }); + }) + .sorted(FileRewriteGroup.taskComparator(rewriteJobOrder)); + Map groupsInPartition = plan.transformValues(List::size); + int totalGroupCount = groupsInPartition.values().stream().reduce(Integer::sum).orElse(0); + return new RewritePositionDeletePlan( + groups, totalGroupCount, groupsInPartition, writeMaxFileSize()); + } + + @Override + protected Iterable filterFiles(Iterable tasks) { + return Iterables.filter(tasks, this::wronglySized); + } + + @Override + protected Iterable> filterFileGroups( + List> groups) { + return Iterables.filter(groups, this::shouldRewrite); + } + + @Override + protected long defaultTargetFileSize() { + return PropertyUtil.propertyAsLong( + table().properties(), + TableProperties.DELETE_TARGET_FILE_SIZE_BYTES, + TableProperties.DELETE_TARGET_FILE_SIZE_BYTES_DEFAULT); + } + + private StructLikeMap>> planFileGroups() { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES); + CloseableIterable fileTasks = planFiles(deletesTable); + + try { + Types.StructType partitionType = Partitioning.partitionType(deletesTable); + StructLikeMap> fileTasksByPartition = + groupByPartition(partitionType, fileTasks); + return fileTasksByPartition.transformValues( + tasks -> ImmutableList.copyOf(planFileGroups(tasks))); + } finally { + try { + fileTasks.close(); + } catch (IOException io) { + LOG.error("Cannot properly close file iterable while planning for rewrite", io); + } + } + } + + private CloseableIterable planFiles(Table deletesTable) { + PositionDeletesTable.PositionDeletesBatchScan scan = + (PositionDeletesTable.PositionDeletesBatchScan) deletesTable.newBatchScan(); + return CloseableIterable.transform( + scan.baseTableFilter(filter).caseSensitive(caseSensitive).ignoreResiduals().planFiles(), + PositionDeletesScanTask.class::cast); + } + + private StructLikeMap> groupByPartition( + Types.StructType partitionType, Iterable tasks) { + StructLikeMap> filesByPartition = + StructLikeMap.create(partitionType); + + for (PositionDeletesScanTask task : tasks) { + StructLike coerced = coercePartition(task, partitionType); + + List partitionTasks = filesByPartition.get(coerced); + if (partitionTasks == null) { + partitionTasks = Lists.newArrayList(); + } + partitionTasks.add(task); + filesByPartition.put(coerced, partitionTasks); + } + + return filesByPartition; + } + + private RewritePositionDeletesGroup newRewriteGroup( + RewriteExecutionContext ctx, + StructLike partition, + List tasks, + long splitSize, + int numOutputSize) { + ImmutableRewritePositionDeleteFiles.FileGroupInfo info = + ImmutableRewritePositionDeleteFiles.FileGroupInfo.builder() + .globalIndex(ctx.currentGlobalIndex()) + .partitionIndex(ctx.currentPartitionIndex(partition)) + .partition(partition) + .build(); + return new RewritePositionDeletesGroup( + info, Lists.newArrayList(tasks), splitSize, numOutputSize); + } + + private boolean shouldRewrite(List group) { + return enoughInputFiles(group) || enoughContent(group) || tooMuchContent(group); + } + + private static class RewriteExecutionContext { + private final Map partitionIndexMap; + private final AtomicInteger groupIndex; + + private RewriteExecutionContext() { + this.partitionIndexMap = Maps.newConcurrentMap(); + this.groupIndex = new AtomicInteger(1); + } + + private int currentGlobalIndex() { + return groupIndex.getAndIncrement(); + } + + private int currentPartitionIndex(StructLike partition) { + return partitionIndexMap.merge(partition, 1, Integer::sum); + } + } + + private StructLike coercePartition(PositionDeletesScanTask task, Types.StructType partitionType) { + return PartitionUtil.coercePartition(partitionType, task.spec(), task.partition()); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedDataRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedDataRewriter.java index e5b5908804e7..5c9e2321fd82 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedDataRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedDataRewriter.java @@ -41,10 +41,17 @@ public abstract class SizeBasedDataRewriter extends SizeBasedFileRewriterDefaults to Integer.MAX_VALUE, which means this feature is not enabled by default. + * + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link + * RewriteFileGroupPlanner#DELETE_FILE_THRESHOLD}. */ - public static final String DELETE_FILE_THRESHOLD = "delete-file-threshold"; + @Deprecated public static final String DELETE_FILE_THRESHOLD = "delete-file-threshold"; - public static final int DELETE_FILE_THRESHOLD_DEFAULT = Integer.MAX_VALUE; + /** + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link + * RewriteFileGroupPlanner#DELETE_FILE_THRESHOLD_DEFAULT}. + */ + @Deprecated public static final int DELETE_FILE_THRESHOLD_DEFAULT = Integer.MAX_VALUE; private int deleteFileThreshold; diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java new file mode 100644 index 000000000000..edaec5af0f27 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java @@ -0,0 +1,340 @@ +/* + * 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.math.RoundingMode; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.math.LongMath; +import org.apache.iceberg.util.BinPacking; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A file rewrite planner that determines which files to rewrite based on their size. + * + *

If files are smaller than the {@link #MIN_FILE_SIZE_BYTES} threshold or larger than the {@link + * #MAX_FILE_SIZE_BYTES} threshold, they are considered targets for being rewritten. + * + *

Once selected, files are grouped based on the {@link BinPacking bin-packing algorithm} into + * groups of no more than {@link #MAX_FILE_GROUP_SIZE_BYTES}. Groups will be actually rewritten if + * they contain more than {@link #MIN_INPUT_FILES} or if they would produce at least one file of + * {@link #TARGET_FILE_SIZE_BYTES}. + * + *

Note that implementations may add extra conditions for selecting files or filtering groups. + */ +public abstract class SizeBasedFileRewritePlanner< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends FileRewriteGroup> + implements FileRewritePlanner { + + private static final Logger LOG = LoggerFactory.getLogger(SizeBasedFileRewritePlanner.class); + + /** The target output file size that this file rewriter will attempt to generate. */ + public static final String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes"; + + /** + * Controls which files will be considered for rewriting. Files with sizes under this threshold + * will be considered for rewriting regardless of any other criteria. + * + *

Defaults to 75% of the target file size. + */ + public static final String MIN_FILE_SIZE_BYTES = "min-file-size-bytes"; + + public static final double MIN_FILE_SIZE_DEFAULT_RATIO = 0.75; + + /** + * Controls which files will be considered for rewriting. Files with sizes above this threshold + * will be considered for rewriting regardless of any other criteria. + * + *

Defaults to 180% of the target file size. + */ + public static final String MAX_FILE_SIZE_BYTES = "max-file-size-bytes"; + + public static final double MAX_FILE_SIZE_DEFAULT_RATIO = 1.80; + + /** + * Any file group exceeding this number of files will be rewritten regardless of other criteria. + * This config ensures file groups that contain many files are compacted even if the total size of + * that group is less than the target file size. This can also be thought of as the maximum number + * of wrongly sized files that could remain in a partition after rewriting. + */ + public static final String MIN_INPUT_FILES = "min-input-files"; + + public static final int MIN_INPUT_FILES_DEFAULT = 5; + + /** Overrides other options and forces rewriting of all provided files. */ + public static final String REWRITE_ALL = "rewrite-all"; + + public static final boolean REWRITE_ALL_DEFAULT = false; + + /** + * This option controls the largest amount of data that should be rewritten in a single file + * group. It helps with breaking down the rewriting of very large partitions which may not be + * rewritable otherwise due to the resource constraints of the cluster. For example, a sort-based + * rewrite may not scale to TB-sized partitions, and those partitions need to be worked on in + * small subsections to avoid exhaustion of resources. + */ + public static final String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes"; + + public static final long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 100L * 1024 * 1024 * 1024; // 100 GB + + private static final long SPLIT_OVERHEAD = 5L * 1024; + + private final Table table; + private long targetFileSize; + private long minFileSize; + private long maxFileSize; + private int minInputFiles; + private boolean rewriteAll; + private long maxGroupSize; + private int outputSpecId; + + protected SizeBasedFileRewritePlanner(Table table) { + this.table = table; + } + + /** Expected target file size before configuration. */ + protected abstract long defaultTargetFileSize(); + + /** Additional filter for tasks before grouping. */ + protected abstract Iterable filterFiles(Iterable tasks); + + /** Additional filter for groups. */ + protected abstract Iterable> filterFileGroups(List> groups); + + @Override + public Set validOptions() { + return ImmutableSet.of( + TARGET_FILE_SIZE_BYTES, + MIN_FILE_SIZE_BYTES, + MAX_FILE_SIZE_BYTES, + MIN_INPUT_FILES, + REWRITE_ALL, + MAX_FILE_GROUP_SIZE_BYTES); + } + + @Override + public void init(Map options) { + Map sizeThresholds = sizeThresholds(options); + this.targetFileSize = sizeThresholds.get(TARGET_FILE_SIZE_BYTES); + this.minFileSize = sizeThresholds.get(MIN_FILE_SIZE_BYTES); + this.maxFileSize = sizeThresholds.get(MAX_FILE_SIZE_BYTES); + this.minInputFiles = minInputFiles(options); + this.rewriteAll = rewriteAll(options); + this.maxGroupSize = maxGroupSize(options); + this.outputSpecId = outputSpecId(options); + + if (rewriteAll) { + LOG.info("Configured to rewrite all provided files in table {}", table.name()); + } + } + + protected Table table() { + return table; + } + + protected boolean wronglySized(T task) { + return task.length() < minFileSize || task.length() > maxFileSize; + } + + protected Iterable> planFileGroups(Iterable tasks) { + Iterable filteredTasks = rewriteAll ? tasks : filterFiles(tasks); + BinPacking.ListPacker packer = new BinPacking.ListPacker<>(maxGroupSize, 1, false); + List> groups = packer.pack(filteredTasks, ContentScanTask::length); + return rewriteAll ? groups : filterFileGroups(groups); + } + + protected boolean enoughInputFiles(List group) { + return group.size() > 1 && group.size() >= minInputFiles; + } + + protected boolean enoughContent(List group) { + return group.size() > 1 && inputSize(group) > targetFileSize; + } + + protected boolean tooMuchContent(List group) { + return inputSize(group) > maxFileSize; + } + + protected long inputSize(List group) { + return group.stream().mapToLong(ContentScanTask::length).sum(); + } + + /** + * Calculates the split size to use in bin-packing rewrites. + * + *

This method determines the target split size as the input size divided by the desired number + * of output files. The final split size is adjusted to be at least as big as the target file size + * but less than the max write file size. + */ + protected long splitSize(long inputSize) { + long estimatedSplitSize = (inputSize / numOutputFiles(inputSize)) + SPLIT_OVERHEAD; + if (estimatedSplitSize < targetFileSize) { + return targetFileSize; + } else { + return Math.min(estimatedSplitSize, writeMaxFileSize()); + } + } + + /** + * Determines the preferable number of output files when rewriting a particular file group. + * + *

If the rewriter is handling 10.1 GB of data with a target file size of 1 GB, it could + * produce 11 files, one of which would only have 0.1 GB. This would most likely be less + * preferable to 10 files with 1.01 GB each. So this method decides whether to round up or round + * down based on what the estimated average file size will be if the remainder (0.1 GB) is + * distributed amongst other files. If the new average file size is no more than 10% greater than + * the target file size, then this method will round down when determining the number of output + * files. Otherwise, the remainder will be written into a separate file. + * + * @param inputSize a total input size for a file group + * @return the number of files this rewriter should create + */ + protected int numOutputFiles(long inputSize) { + if (inputSize < targetFileSize) { + return 1; + } + + long numFilesWithRemainder = LongMath.divide(inputSize, targetFileSize, RoundingMode.CEILING); + long numFilesWithoutRemainder = LongMath.divide(inputSize, targetFileSize, RoundingMode.FLOOR); + long avgFileSizeWithoutRemainder = inputSize / numFilesWithoutRemainder; + + if (LongMath.mod(inputSize, targetFileSize) > minFileSize) { + // the remainder file is of a valid size for this rewrite so keep it + return (int) numFilesWithRemainder; + + } else if (avgFileSizeWithoutRemainder < Math.min(1.1 * targetFileSize, writeMaxFileSize())) { + // if the reminder is distributed amongst other files, + // the average file size will be no more than 10% bigger than the target file size + // so round down and distribute remainder amongst other files + return (int) numFilesWithoutRemainder; + + } else { + // keep the remainder file as it is not OK to distribute it amongst other files + return (int) numFilesWithRemainder; + } + } + + /** + * Estimates a larger max target file size than the target size used in task creation to avoid + * creating tiny remainder files. + * + *

While we create tasks that should all be smaller than our target size, there is a chance + * that the actual data will end up being larger than our target size due to various factors of + * compression, serialization, which are outside our control. If this occurs, instead of making a + * single file that is close in size to our target, we would end up producing one file of the + * target size, and then a small extra file with the remaining data. + * + *

For example, if our target is 512 MB, we may generate a rewrite task that should be 500 MB. + * When we write the data we may find we actually have to write out 530 MB. If we use the target + * size while writing, we would produce a 512 MB file and an 18 MB file. If instead we use a + * larger size estimated by this method, then we end up writing a single file. + * + * @return the target size plus one half of the distance between max and target + */ + protected long writeMaxFileSize() { + return (long) (targetFileSize + ((maxFileSize - targetFileSize) * 0.5)); + } + + protected int outputSpecId() { + return outputSpecId; + } + + private int outputSpecId(Map options) { + int specId = + PropertyUtil.propertyAsInt(options, RewriteDataFiles.OUTPUT_SPEC_ID, table.spec().specId()); + Preconditions.checkArgument( + table.specs().containsKey(specId), + "Cannot use output spec id %s because the table does not contain a reference to this spec-id.", + specId); + return specId; + } + + private Map sizeThresholds(Map options) { + long target = + PropertyUtil.propertyAsLong(options, TARGET_FILE_SIZE_BYTES, defaultTargetFileSize()); + + long defaultMin = (long) (target * MIN_FILE_SIZE_DEFAULT_RATIO); + long min = PropertyUtil.propertyAsLong(options, MIN_FILE_SIZE_BYTES, defaultMin); + + long defaultMax = (long) (target * MAX_FILE_SIZE_DEFAULT_RATIO); + long max = PropertyUtil.propertyAsLong(options, MAX_FILE_SIZE_BYTES, defaultMax); + + Preconditions.checkArgument( + target > 0, "'%s' is set to %s but must be > 0", TARGET_FILE_SIZE_BYTES, target); + + Preconditions.checkArgument( + min >= 0, "'%s' is set to %s but must be >= 0", MIN_FILE_SIZE_BYTES, min); + + Preconditions.checkArgument( + target > min, + "'%s' (%s) must be > '%s' (%s), all new files will be smaller than the min threshold", + TARGET_FILE_SIZE_BYTES, + target, + MIN_FILE_SIZE_BYTES, + min); + + Preconditions.checkArgument( + target < max, + "'%s' (%s) must be < '%s' (%s), all new files will be larger than the max threshold", + TARGET_FILE_SIZE_BYTES, + target, + MAX_FILE_SIZE_BYTES, + max); + + Map values = Maps.newHashMap(); + + values.put(TARGET_FILE_SIZE_BYTES, target); + values.put(MIN_FILE_SIZE_BYTES, min); + values.put(MAX_FILE_SIZE_BYTES, max); + + return values; + } + + private int minInputFiles(Map options) { + int value = PropertyUtil.propertyAsInt(options, MIN_INPUT_FILES, MIN_INPUT_FILES_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", MIN_INPUT_FILES, value); + return value; + } + + private long maxGroupSize(Map options) { + long value = + PropertyUtil.propertyAsLong( + options, MAX_FILE_GROUP_SIZE_BYTES, MAX_FILE_GROUP_SIZE_BYTES_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", MAX_FILE_GROUP_SIZE_BYTES, value); + return value; + } + + private boolean rewriteAll(Map options) { + return PropertyUtil.propertyAsBoolean(options, REWRITE_ALL, REWRITE_ALL_DEFAULT); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java index cea7003c1a38..00ef0b6694de 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java @@ -47,7 +47,11 @@ * {@link #TARGET_FILE_SIZE_BYTES}. * *

Note that implementations may add extra conditions for selecting files or filtering groups. + * + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link SizeBasedFileRewritePlanner} and + * {@link FileRewriteExecutor}. */ +@Deprecated public abstract class SizeBasedFileRewriter, F extends ContentFile> implements FileRewriter { diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedPositionDeletesRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedPositionDeletesRewriter.java index c08a31a731f4..60f37b79d24c 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedPositionDeletesRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedPositionDeletesRewriter.java @@ -26,6 +26,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.PropertyUtil; +/** + * @deprecated since 1.8.0, will be removed in 1.9.0; use {@link RewritePositionDeletesGroupPlanner} + * and {@link FileRewriteExecutor}. + */ +@Deprecated public abstract class SizeBasedPositionDeletesRewriter extends SizeBasedFileRewriter { diff --git a/core/src/test/java/org/apache/iceberg/actions/TestRewriteFileGroupPlanner.java b/core/src/test/java/org/apache/iceberg/actions/TestRewriteFileGroupPlanner.java new file mode 100644 index 000000000000..48991cd601b7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/actions/TestRewriteFileGroupPlanner.java @@ -0,0 +1,390 @@ +/* + * 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 static org.apache.iceberg.actions.RewriteDataFiles.REWRITE_JOB_ORDER; +import static org.apache.iceberg.actions.RewriteFileGroupPlanner.MAX_FILE_SIZE_DEFAULT_RATIO; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.RewriteJobOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; + +class TestRewriteFileGroupPlanner { + private static final Map REWRITE_ALL = + ImmutableMap.of(RewriteFileGroupPlanner.REWRITE_ALL, "true"); + + private static final DataFile FILE_1 = newDataFile("data_bucket=0", 10); + private static final DataFile FILE_2 = newDataFile("data_bucket=0", 10); + private static final DataFile FILE_3 = newDataFile("data_bucket=0", 10); + private static final DataFile FILE_4 = newDataFile("data_bucket=1", 11); + private static final DataFile FILE_5 = newDataFile("data_bucket=1", 11); + private static final DataFile FILE_6 = newDataFile("data_bucket=2", 50); + + private static final Map> EXPECTED = + ImmutableMap.of( + RewriteJobOrder.FILES_DESC, + ImmutableList.of(FILE_1.partition(), FILE_4.partition(), FILE_6.partition()), + RewriteJobOrder.FILES_ASC, + ImmutableList.of(FILE_6.partition(), FILE_4.partition(), FILE_1.partition()), + RewriteJobOrder.BYTES_DESC, + ImmutableList.of(FILE_6.partition(), FILE_1.partition(), FILE_4.partition()), + RewriteJobOrder.BYTES_ASC, + ImmutableList.of(FILE_4.partition(), FILE_1.partition(), FILE_6.partition())); + + @TempDir private File tableDir = null; + private TestTables.TestTable table = null; + + @BeforeEach + public void setupTable() throws Exception { + this.table = TestTables.create(tableDir, "test", TestBase.SCHEMA, TestBase.SPEC, 3); + } + + @AfterEach + public void cleanupTables() { + TestTables.clearTables(); + } + + @ParameterizedTest + @EnumSource( + value = RewriteJobOrder.class, + names = {"FILES_DESC", "FILES_ASC", "BYTES_DESC", "BYTES_ASC"}) + void testJobOrder(RewriteJobOrder order) { + addFiles(); + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.REWRITE_ALL, "true", REWRITE_JOB_ORDER, order.name())); + RewriteFilePlan result = planner.plan(); + List groups = result.groups().collect(Collectors.toList()); + assertThat(groups.stream().map(group -> group.info().partition()).collect(Collectors.toList())) + .isEqualTo(EXPECTED.get(order)); + assertThat(result.totalGroupCount()).isEqualTo(3); + EXPECTED.get(order).forEach(s -> assertThat(result.groupsInPartition(s)).isEqualTo(1)); + } + + @Test + void testUnpartitionedTable() { + table.updateSpec().removeField("data_bucket").commit(); + table.refresh(); + + table + .newAppend() + .appendFile(newDataFile("", 10)) + .appendFile(newDataFile("", 20)) + .appendFile(newDataFile("", 30)) + .commit(); + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_INPUT_FILES, + "1", + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, + "30")); + RewriteFilePlan result = planner.plan(); + assertThat(result.totalGroupCount()).isEqualTo(1); + assertThat(result.groups().iterator().next().numInputFiles()).isEqualTo(2); + } + + @Test + void testMaxGroupSize() { + addFiles(); + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.REWRITE_ALL, + "true", + RewriteFileGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES, + "10")); + RewriteFilePlan result = planner.plan(); + assertThat(result.totalGroupCount()).isEqualTo(6); + assertThat(result.groupsInPartition(FILE_1.partition())).isEqualTo(3); + assertThat(result.groupsInPartition(FILE_4.partition())).isEqualTo(2); + assertThat(result.groupsInPartition(FILE_6.partition())).isEqualTo(1); + } + + @Test + void testEmptyTable() { + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + planner.init(REWRITE_ALL); + + RewriteFilePlan result = planner.plan(); + + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + assertThat(result.totalGroupCount()).isZero(); + } + + @Test + void testFilter() { + addFiles(); + RewriteFileGroupPlanner planner = + new RewriteFileGroupPlanner( + table, + Expressions.or( + Expressions.equal(Expressions.bucket("data", 16), 0), + Expressions.equal(Expressions.bucket("data", 16), 2))); + planner.init(REWRITE_ALL); + RewriteFilePlan plan = planner.plan(); + List groups = plan.groups().collect(Collectors.toList()); + + assertThat(plan.totalGroupCount()).isEqualTo(2); + assertThat(groups).hasSize(2); + assertThat(groups.stream().mapToLong(FileRewriteGroup::numInputFiles).sum()).isEqualTo(4); + } + + @Test + void testWriteMaxFileSize() { + int targetFileSize = 10; + addFiles(); + + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.REWRITE_ALL, + "true", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, + String.valueOf(targetFileSize))); + RewriteFilePlan plan = planner.plan(); + assertThat(plan.writeMaxFileSize()) + .isGreaterThan(targetFileSize) + .isLessThan((long) (targetFileSize * MAX_FILE_SIZE_DEFAULT_RATIO)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testOutputSpec(boolean specific) { + addFiles(); + + int oldSpecId = table.spec().specId(); + table.updateSpec().removeField("data_bucket").commit(); + table.newAppend().appendFile(newDataFile("", 10)).commit(); + table.refresh(); + int newSpecId = table.spec().specId(); + + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + Map options = Maps.newHashMap(REWRITE_ALL); + if (specific) { + options.put(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(oldSpecId)); + } + + planner.init(options); + + RewriteFilePlan plan = planner.plan(); + assertThat(plan.outputSpecId()).isEqualTo(specific ? oldSpecId : newSpecId); + } + + @Test + void testValidOptions() { + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_INPUT_FILES, + RewriteFileGroupPlanner.REWRITE_ALL, + RewriteFileGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES, + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, + RewriteDataFiles.REWRITE_JOB_ORDER)); + } + + @Test + void testInvalidOption() { + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + Map invalidRewriteJobOrderOptions = + ImmutableMap.of(RewriteDataFiles.REWRITE_JOB_ORDER, "foo"); + assertThatThrownBy(() -> planner.init(invalidRewriteJobOrderOptions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid rewrite job order name: foo"); + + Map invalidOutputSpecIdOptions = + ImmutableMap.of(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(1234)); + assertThatThrownBy(() -> planner.init(invalidOutputSpecIdOptions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot use output spec id 1234 because the table does not contain a reference to this spec-id."); + + Map invalidDeleteFileThresholdOptions = + ImmutableMap.of(RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, "-1"); + assertThatThrownBy(() -> planner.init(invalidDeleteFileThresholdOptions)) + .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); + } + + @Test + void testBinPackDataSelectFiles() { + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + checkDataFileSizeFiltering(planner); + checkDataFilesDeleteThreshold(planner); + checkDataFileGroupWithEnoughFiles(planner); + checkDataFileGroupWithEnoughData(planner); + checkDataFileGroupWithTooMuchData(planner); + } + + private void checkDataFileSizeFiltering(RewriteFileGroupPlanner planner) { + FileScanTask tooSmallTask = new MockFileScanTask(100L); + FileScanTask optimal = new MockFileScanTask(450); + FileScanTask tooBigTask = new MockFileScanTask(1000L); + List tasks = ImmutableList.of(tooSmallTask, optimal, tooBigTask); + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, "250", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, "500", + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, "750", + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); + planner.init(options); + + Iterable> groups = planner.planFileGroups(tasks); + assertThat(groups).as("Must have 1 group").hasSize(1); + List group = Iterables.getOnlyElement(groups); + assertThat(group).as("Must rewrite 2 files").hasSize(2); + } + + private void checkDataFilesDeleteThreshold(RewriteFileGroupPlanner planner) { + FileScanTask tooManyDeletesTask = MockFileScanTask.mockTaskWithDeletes(1000L, 3); + FileScanTask optimalTask = MockFileScanTask.mockTaskWithDeletes(1000L, 1); + List tasks = ImmutableList.of(tooManyDeletesTask, optimalTask); + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, "1", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, "2000", + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, "5000", + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, "2"); + planner.init(options); + + Iterable> groups = planner.planFileGroups(tasks); + assertThat(groups).as("Must have 1 group").hasSize(1); + List group = Iterables.getOnlyElement(groups); + assertThat(group).as("Must rewrite 1 file").hasSize(1); + } + + private void checkDataFileGroupWithEnoughFiles(RewriteFileGroupPlanner planner) { + List tasks = + ImmutableList.of( + new MockFileScanTask(100L), + new MockFileScanTask(100L), + new MockFileScanTask(100L), + new MockFileScanTask(100L)); + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_INPUT_FILES, "3", + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, "150", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, "1000", + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, "5000", + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); + planner.init(options); + + Iterable> groups = planner.planFileGroups(tasks); + assertThat(groups).as("Must have 1 group").hasSize(1); + List group = Iterables.getOnlyElement(groups); + assertThat(group).as("Must rewrite 4 files").hasSize(4); + } + + private void checkDataFileGroupWithEnoughData(RewriteFileGroupPlanner planner) { + List tasks = + ImmutableList.of( + new MockFileScanTask(100L), new MockFileScanTask(100L), new MockFileScanTask(100L)); + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_INPUT_FILES, "5", + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, "200", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, "250", + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, "500", + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); + planner.init(options); + + Iterable> groups = planner.planFileGroups(tasks); + assertThat(groups).as("Must have 1 group").hasSize(1); + List group = Iterables.getOnlyElement(groups); + assertThat(group).as("Must rewrite 3 files").hasSize(3); + } + + private void checkDataFileGroupWithTooMuchData(RewriteFileGroupPlanner planner) { + List tasks = ImmutableList.of(new MockFileScanTask(2000L)); + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_INPUT_FILES, "5", + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, "200", + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, "250", + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, "500", + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); + planner.init(options); + + Iterable> groups = planner.planFileGroups(tasks); + assertThat(groups).as("Must have 1 group").hasSize(1); + List group = Iterables.getOnlyElement(groups); + assertThat(group).as("Must rewrite big file").hasSize(1); + } + + private void addFiles() { + table + .newAppend() + .appendFile(FILE_1) + .appendFile(FILE_2) + .appendFile(FILE_3) + .appendFile(FILE_4) + .appendFile(FILE_5) + .appendFile(FILE_6) + .commit(); + } + + private static DataFile newDataFile(String partitionPath, long fileSize) { + return DataFiles.builder(TestBase.SPEC) + .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(fileSize) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/actions/TestRewritePositionDeletesGroupPlanner.java b/core/src/test/java/org/apache/iceberg/actions/TestRewritePositionDeletesGroupPlanner.java new file mode 100644 index 000000000000..07858706d9ee --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/actions/TestRewritePositionDeletesGroupPlanner.java @@ -0,0 +1,264 @@ +/* + * 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 static org.apache.iceberg.actions.RewritePositionDeleteFiles.REWRITE_JOB_ORDER; +import static org.apache.iceberg.actions.RewritePositionDeletesGroupPlanner.MAX_FILE_SIZE_DEFAULT_RATIO; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RewriteJobOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +class TestRewritePositionDeletesGroupPlanner { + private static final Map REWRITE_ALL = + ImmutableMap.of(RewritePositionDeletesGroupPlanner.REWRITE_ALL, "true"); + + private static final DataFile FILE_1 = newDataFile("data_bucket=0"); + private static final DataFile FILE_2 = newDataFile("data_bucket=1"); + private static final DataFile FILE_3 = newDataFile("data_bucket=2"); + private static final Map> EXPECTED = + ImmutableMap.of( + RewriteJobOrder.FILES_DESC, + ImmutableList.of(FILE_1.partition(), FILE_2.partition(), FILE_3.partition()), + RewriteJobOrder.FILES_ASC, + ImmutableList.of(FILE_3.partition(), FILE_2.partition(), FILE_1.partition()), + RewriteJobOrder.BYTES_DESC, + ImmutableList.of(FILE_3.partition(), FILE_1.partition(), FILE_2.partition()), + RewriteJobOrder.BYTES_ASC, + ImmutableList.of(FILE_2.partition(), FILE_1.partition(), FILE_3.partition())); + + @TempDir private File tableDir = null; + private TestTables.TestTable table = null; + + @BeforeEach + public void setupTable() throws Exception { + this.table = TestTables.create(tableDir, "test", TestBase.SCHEMA, TestBase.SPEC, 2); + } + + @AfterEach + public void cleanupTables() { + TestTables.clearTables(); + } + + @ParameterizedTest + @EnumSource( + value = RewriteJobOrder.class, + names = {"FILES_DESC", "FILES_ASC", "BYTES_DESC", "BYTES_ASC"}) + void testJobOrder(RewriteJobOrder order) { + addFiles(); + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.REWRITE_ALL, "true", REWRITE_JOB_ORDER, order.name())); + RewritePositionDeletePlan result = planner.plan(); + List groups = result.groups().collect(Collectors.toList()); + assertThat( + groups.stream() + .map( + group -> + new PartitionData(TestBase.SPEC.partitionType()) + .copyFor(group.info().partition())) + .collect(Collectors.toList())) + .isEqualTo(EXPECTED.get(order)); + assertThat(result.totalGroupCount()).isEqualTo(3); + EXPECTED.get(order).forEach(s -> assertThat(result.groupsInPartition(s)).isEqualTo(1)); + } + + @Test + void testUnpartitionedTable() { + table.updateSpec().removeField("data_bucket").commit(); + table.refresh(); + + table + .newRowDelta() + .addRows(newDataFile("")) + .addDeletes(newDeleteFile(10)) + .addDeletes(newDeleteFile(20)) + .addDeletes(newDeleteFile(30)) + .commit(); + + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_INPUT_FILES, + "1", + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, + "30")); + RewritePositionDeletePlan result = planner.plan(); + assertThat(result.totalGroupCount()).isEqualTo(1); + assertThat(result.groups().iterator().next().numInputFiles()).isEqualTo(2); + } + + @Test + void testMaxGroupSize() { + addFiles(); + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewritePositionDeletesGroupPlanner.REWRITE_ALL, + "true", + RewritePositionDeletesGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES, + "10")); + RewritePositionDeletePlan result = planner.plan(); + assertThat(result.totalGroupCount()).isEqualTo(6); + assertThat(result.groupsInPartition(FILE_1.partition())).isEqualTo(3); + assertThat(result.groupsInPartition(FILE_2.partition())).isEqualTo(2); + assertThat(result.groupsInPartition(FILE_3.partition())).isEqualTo(1); + } + + @Test + void testEmptyTable() { + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + + planner.init(REWRITE_ALL); + + RewritePositionDeletePlan result = planner.plan(); + + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + assertThat(result.totalGroupCount()).isZero(); + } + + @Test + void testFilter() { + addFiles(); + RewritePositionDeletesGroupPlanner planner = + new RewritePositionDeletesGroupPlanner( + table, + Expressions.or( + Expressions.equal(Expressions.bucket("data", 16), 0), + Expressions.equal(Expressions.bucket("data", 16), 2)), + false); + planner.init(REWRITE_ALL); + RewritePositionDeletePlan plan = planner.plan(); + List groups = plan.groups().collect(Collectors.toList()); + + assertThat(plan.totalGroupCount()).isEqualTo(2); + assertThat(groups).hasSize(2); + assertThat(groups.stream().mapToLong(FileRewriteGroup::numInputFiles).sum()).isEqualTo(4); + } + + @Test + void testWriteMaxFileSize() { + int targetFileSize = 10; + addFiles(); + + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + planner.init( + ImmutableMap.of( + RewritePositionDeletesGroupPlanner.REWRITE_ALL, + "true", + RewritePositionDeletesGroupPlanner.TARGET_FILE_SIZE_BYTES, + String.valueOf(targetFileSize))); + RewritePositionDeletePlan plan = planner.plan(); + assertThat(plan.writeMaxFileSize()) + .isGreaterThan(targetFileSize) + .isLessThan((long) (targetFileSize * MAX_FILE_SIZE_DEFAULT_RATIO)); + } + + @Test + void testValidOptions() { + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + RewritePositionDeletesGroupPlanner.TARGET_FILE_SIZE_BYTES, + RewritePositionDeletesGroupPlanner.MIN_FILE_SIZE_BYTES, + RewritePositionDeletesGroupPlanner.MAX_FILE_SIZE_BYTES, + RewritePositionDeletesGroupPlanner.MIN_INPUT_FILES, + RewritePositionDeletesGroupPlanner.REWRITE_ALL, + RewritePositionDeletesGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES, + RewriteDataFiles.REWRITE_JOB_ORDER)); + } + + @Test + void testInvalidOption() { + RewritePositionDeletesGroupPlanner planner = new RewritePositionDeletesGroupPlanner(table); + + Map invalidRewriteJobOrderOptions = + ImmutableMap.of(RewritePositionDeleteFiles.REWRITE_JOB_ORDER, "foo"); + assertThatThrownBy(() -> planner.init(invalidRewriteJobOrderOptions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid rewrite job order name: foo"); + } + + private void addFiles() { + table + .newRowDelta() + .addRows(FILE_1) + .addDeletes(newDeleteFile(FILE_1.partition(), 10)) + .addDeletes(newDeleteFile(FILE_1.partition(), 10)) + .addDeletes(newDeleteFile(FILE_1.partition(), 10)) + .addRows(FILE_2) + .addDeletes(newDeleteFile(FILE_2.partition(), 11)) + .addDeletes(newDeleteFile(FILE_2.partition(), 11)) + .addRows(FILE_3) + .addDeletes(newDeleteFile(FILE_3.partition(), 50)) + .commit(); + } + + private static DataFile newDataFile(String partitionPath) { + return DataFiles.builder(TestBase.SPEC) + .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); + } + + private static DeleteFile newDeleteFile(long fileSize) { + return newDeleteFile( + new PartitionData(PartitionSpec.unpartitioned().partitionType()), fileSize); + } + + private static DeleteFile newDeleteFile(StructLike partition, long fileSize) { + return FileMetadata.deleteFileBuilder(TestBase.SPEC) + .ofPositionDeletes() + .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(fileSize) + .withPartition(partition) + .withRecordCount(1) + .build(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedFileRewritePlanner.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedFileRewritePlanner.java new file mode 100644 index 000000000000..43ea307e11ff --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedFileRewritePlanner.java @@ -0,0 +1,185 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; + +class TestSizeBasedFileRewritePlanner { + @TempDir private File tableDir = null; + private TestTables.TestTable table = null; + + @BeforeEach + public void setupTable() throws Exception { + this.table = TestTables.create(tableDir, "test", TestBase.SCHEMA, TestBase.SPEC, 3); + } + + @AfterEach + public void cleanupTables() { + TestTables.clearTables(); + } + + @Test + void testSplitSizeLowerBound() { + FileScanTask task1 = new MockFileScanTask(mockDataFile()); + FileScanTask task2 = new MockFileScanTask(mockDataFile()); + FileScanTask task3 = new MockFileScanTask(mockDataFile()); + FileScanTask task4 = new MockFileScanTask(mockDataFile()); + List tasks = ImmutableList.of(task1, task2, task3, task4); + + TestingPlanner planner = new TestingPlanner(table); + + long minFileSize = 256L * 1024 * 1024; + long targetFileSize = 512L * 1024 * 1024; + long maxFileSize = 768L * 1024 * 1024; + + Map options = + ImmutableMap.of( + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, String.valueOf(minFileSize), + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, String.valueOf(targetFileSize), + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, String.valueOf(maxFileSize)); + planner.init(options); + + // the total task size is 580 MB and the target file size is 512 MB + // the remainder must be written into a separate file as it exceeds 10% + List> groups = Lists.newArrayList(planner.planFileGroups(tasks).iterator()); + + assertThat(groups).hasSize(1); + + List group = groups.get(0); + // the split size must be >= targetFileSize and < maxFileSize + long splitSize = group.stream().mapToLong(FileScanTask::sizeBytes).sum(); + assertThat(splitSize).isGreaterThanOrEqualTo(targetFileSize).isLessThan(maxFileSize); + } + + @Test + void testValidOptions() { + TestingPlanner planner = new TestingPlanner(table); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_INPUT_FILES, + RewriteFileGroupPlanner.REWRITE_ALL, + RewriteFileGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES)); + } + + @Test + void testInvalidOption() { + TestingPlanner planner = new TestingPlanner(table); + + Map invalidTargetSizeOptions = + ImmutableMap.of(SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, "0"); + assertThatThrownBy(() -> planner.init(invalidTargetSizeOptions)) + .hasMessageContaining("'target-file-size-bytes' is set to 0 but must be > 0"); + + Map invalidMinSizeOptions = + ImmutableMap.of(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "-1"); + assertThatThrownBy(() -> planner.init(invalidMinSizeOptions)) + .hasMessageContaining("'min-file-size-bytes' is set to -1 but must be >= 0"); + + Map invalidTargetMinSizeOptions = + ImmutableMap.of( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, "3", + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "5"); + assertThatThrownBy(() -> planner.init(invalidTargetMinSizeOptions)) + .hasMessageContaining("'target-file-size-bytes' (3) must be > 'min-file-size-bytes' (5)") + .hasMessageContaining("all new files will be smaller than the min threshold"); + + Map invalidTargetMaxSizeOptions = + ImmutableMap.of( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, "5", + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, "3"); + assertThatThrownBy(() -> planner.init(invalidTargetMaxSizeOptions)) + .hasMessageContaining("'target-file-size-bytes' (5) must be < 'max-file-size-bytes' (3)") + .hasMessageContaining("all new files will be larger than the max threshold"); + + Map invalidMinInputFilesOptions = + ImmutableMap.of(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "0"); + assertThatThrownBy(() -> planner.init(invalidMinInputFilesOptions)) + .hasMessageContaining("'min-input-files' is set to 0 but must be > 0"); + + Map invalidMaxFileGroupSizeOptions = + ImmutableMap.of(SizeBasedFileRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES, "0"); + assertThatThrownBy(() -> planner.init(invalidMaxFileGroupSizeOptions)) + .hasMessageContaining("'max-file-group-size-bytes' is set to 0 but must be > 0"); + } + + private static class TestingPlanner + extends SizeBasedFileRewritePlanner< + RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, RewriteFileGroup> { + protected TestingPlanner(Table table) { + super(table); + } + + @Override + protected long defaultTargetFileSize() { + return TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; + } + + @Override + protected Iterable filterFiles(Iterable tasks) { + return tasks; + } + + @Override + protected Iterable> filterFileGroups(List> groups) { + return groups; + } + + @Override + public FileRewritePlan + plan() { + throw new UnsupportedOperationException("Not supported"); + } + } + + private DataFile mockDataFile() { + DataFile file = Mockito.mock(DataFile.class); + when(file.partition()).thenReturn(Mockito.mock(StructLike.class)); + when(file.fileSizeInBytes()).thenReturn(145L * 1024 * 1024); + return file; + } +} diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java deleted file mode 100644 index 77d16d3bc821..000000000000 --- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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 static org.assertj.core.api.Assertions.assertThat; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MockFileScanTask; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestSizeBasedRewriter extends TestBase { - - @Parameters(name = "formatVersion = {0}") - protected static List parameters() { - return Arrays.asList(1, 2, 3); - } - - @TestTemplate - public void testSplitSizeLowerBound() { - SizeBasedDataFileRewriterImpl rewriter = new SizeBasedDataFileRewriterImpl(table); - - FileScanTask task1 = new MockFileScanTask(145L * 1024 * 1024); - FileScanTask task2 = new MockFileScanTask(145L * 1024 * 1024); - FileScanTask task3 = new MockFileScanTask(145L * 1024 * 1024); - FileScanTask task4 = new MockFileScanTask(145L * 1024 * 1024); - List tasks = ImmutableList.of(task1, task2, task3, task4); - - long minFileSize = 256L * 1024 * 1024; - long targetFileSize = 512L * 1024 * 1024; - long maxFileSize = 768L * 1024 * 1024; - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, String.valueOf(minFileSize), - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, String.valueOf(targetFileSize), - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, String.valueOf(maxFileSize)); - rewriter.init(options); - - // the total task size is 580 MB and the target file size is 512 MB - // the remainder must be written into a separate file as it exceeds 10% - long numOutputFiles = rewriter.computeNumOutputFiles(tasks); - assertThat(numOutputFiles).isEqualTo(2); - - // the split size must be >= targetFileSize and < maxFileSize - long splitSize = rewriter.computeSplitSize(tasks); - assertThat(splitSize).isGreaterThanOrEqualTo(targetFileSize); - assertThat(splitSize).isLessThan(maxFileSize); - } - - private static class SizeBasedDataFileRewriterImpl extends SizeBasedDataRewriter { - - SizeBasedDataFileRewriterImpl(Table table) { - super(table); - } - - @Override - public Set rewrite(List group) { - throw new UnsupportedOperationException("Not implemented"); - } - - public long computeSplitSize(List group) { - return splitSize(inputSize(group)); - } - - public long computeNumOutputFiles(List group) { - return numOutputFiles(inputSize(group)); - } - } -} diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index f3be0a870972..5a1bdb983f7c 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -51,7 +51,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; -import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.PositionDelete; @@ -217,7 +217,7 @@ private void testDanglingDelete(String partitionCol, int numDataFiles) throws Ex SparkActions.get(spark) .rewriteDataFiles(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); // write dangling delete files for 'old data files' @@ -230,7 +230,7 @@ private void testDanglingDelete(String partitionCol, int numDataFiles) throws Ex Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java index 95bebc7caed4..88ab82bd600a 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java @@ -35,7 +35,7 @@ import org.apache.iceberg.SortDirection; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; -import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.relocated.com.google.common.io.Files; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -105,7 +105,7 @@ public void cleanUpIteration() throws IOException { public void sortInt() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -118,7 +118,7 @@ public void sortInt() { public void sortInt2() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -132,7 +132,7 @@ public void sortInt2() { public void sortInt3() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -148,7 +148,7 @@ public void sortInt3() { public void sortInt4() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -164,7 +164,7 @@ public void sortInt4() { public void sortString() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -177,7 +177,7 @@ public void sortString() { public void sortFourColumns() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -193,7 +193,7 @@ public void sortFourColumns() { public void sortSixColumns() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort( SortOrder.builderFor(table().schema()) .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) @@ -211,7 +211,7 @@ public void sortSixColumns() { public void zSortInt() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("intCol") .execute(); } @@ -221,7 +221,7 @@ public void zSortInt() { public void zSortInt2() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("intCol", "intCol2") .execute(); } @@ -231,7 +231,7 @@ public void zSortInt2() { public void zSortInt3() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("intCol", "intCol2", "intCol3") .execute(); } @@ -241,7 +241,7 @@ public void zSortInt3() { public void zSortInt4() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("intCol", "intCol2", "intCol3", "intCol4") .execute(); } @@ -251,7 +251,7 @@ public void zSortInt4() { public void zSortString() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("stringCol") .execute(); } @@ -261,7 +261,7 @@ public void zSortString() { public void zSortFourColumns() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("stringCol", "intCol", "dateCol", "doubleCol") .execute(); } @@ -271,7 +271,7 @@ public void zSortFourColumns() { public void zSortSixColumns() { SparkActions.get() .rewriteDataFiles(table()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("stringCol", "intCol", "dateCol", "timestampCol", "doubleCol", "longCol") .execute(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java index e04a0c88b4bb..29585a85ac59 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -18,54 +18,45 @@ */ package org.apache.iceberg.spark.actions; -import java.io.IOException; import java.math.RoundingMode; import java.util.Arrays; import java.util.Collection; 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.DataFile; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.SortOrder; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; -import org.apache.iceberg.actions.FileRewriter; +import org.apache.iceberg.actions.FileRewriteExecutor; import org.apache.iceberg.actions.ImmutableRewriteDataFiles; import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; -import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.actions.RewriteFileGroupPlanner; +import org.apache.iceberg.actions.RewriteFilePlan; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; -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.Iterables; -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.spark.SparkUtil; -import org.apache.iceberg.types.Types.StructType; 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; @@ -101,9 +92,12 @@ public class RewriteDataFilesSparkAction private boolean partialProgressEnabled; private boolean removeDanglingDeletes; private boolean useStartingSequenceNumber; - private RewriteJobOrder rewriteJobOrder; - private FileRewriter rewriter = null; private boolean caseSensitive; + private RewriteFileGroupPlanner planner = null; + private FileRewriteExecutor< + FileGroupInfo, FileScanTask, DataFile, RewriteFileGroup, RewriteFilePlan> + rewriter = null; + private boolean shufflingPlanner = false; RewriteDataFilesSparkAction(SparkSession spark, Table table) { super(spark.cloneSession()); @@ -122,7 +116,8 @@ protected RewriteDataFilesSparkAction self() { public RewriteDataFilesSparkAction binPack() { Preconditions.checkArgument( rewriter == null, "Must use only one rewriter type (bin-pack, sort, zorder)"); - this.rewriter = new SparkBinPackDataRewriter(spark(), table); + this.rewriter = new SparkBinPackDataRewriteExecutor(spark(), table); + this.shufflingPlanner = false; return this; } @@ -130,7 +125,8 @@ public RewriteDataFilesSparkAction binPack() { public RewriteDataFilesSparkAction sort(SortOrder sortOrder) { Preconditions.checkArgument( rewriter == null, "Must use only one rewriter type (bin-pack, sort, zorder)"); - this.rewriter = new SparkSortDataRewriter(spark(), table, sortOrder); + this.rewriter = new SparkSortDataRewriteExecutor(spark(), table, sortOrder); + this.shufflingPlanner = true; return this; } @@ -138,7 +134,8 @@ public RewriteDataFilesSparkAction sort(SortOrder sortOrder) { public RewriteDataFilesSparkAction sort() { Preconditions.checkArgument( rewriter == null, "Must use only one rewriter type (bin-pack, sort, zorder)"); - this.rewriter = new SparkSortDataRewriter(spark(), table); + this.rewriter = new SparkSortDataRewriteExecutor(spark(), table); + this.shufflingPlanner = true; return this; } @@ -146,7 +143,8 @@ public RewriteDataFilesSparkAction sort() { public RewriteDataFilesSparkAction zOrder(String... columnNames) { Preconditions.checkArgument( rewriter == null, "Must use only one rewriter type (bin-pack, sort, zorder)"); - this.rewriter = new SparkZOrderDataRewriter(spark(), table, Arrays.asList(columnNames)); + this.rewriter = new SparkZOrderDataRewriteExecutor(spark(), table, Arrays.asList(columnNames)); + this.shufflingPlanner = true; return this; } @@ -164,28 +162,20 @@ public RewriteDataFiles.Result execute() { long startingSnapshotId = table.currentSnapshot().snapshotId(); - // Default to BinPack if no strategy selected - if (this.rewriter == null) { - this.rewriter = new SparkBinPackDataRewriter(spark(), table); - } - - validateAndInitOptions(); + init(startingSnapshotId); - StructLikeMap>> fileGroupsByPartition = - planFileGroups(startingSnapshotId); - RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition); + RewriteFilePlan plan = plan(); + rewriter.initPlan(plan); - if (ctx.totalGroupCount() == 0) { + if (plan.totalGroupCount() == 0) { LOG.info("Nothing found to rewrite in {}", table.name()); return EMPTY_RESULT; } - Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - Builder resultBuilder = partialProgressEnabled - ? doExecuteWithPartialProgress(ctx, groupStream, commitManager(startingSnapshotId)) - : doExecute(ctx, groupStream, commitManager(startingSnapshotId)); + ? doExecuteWithPartialProgress(plan, commitManager(startingSnapshotId)) + : doExecute(plan, commitManager(startingSnapshotId)); if (removeDanglingDeletes) { RemoveDanglingDeletesSparkAction action = @@ -193,72 +183,37 @@ public RewriteDataFiles.Result execute() { int removedCount = Iterables.size(action.execute().removedDeleteFiles()); resultBuilder.removedDeleteFilesCount(removedCount); } + return resultBuilder.build(); } - StructLikeMap>> planFileGroups(long startingSnapshotId) { - CloseableIterable fileScanTasks = - table - .newScan() - .useSnapshot(startingSnapshotId) - .caseSensitive(caseSensitive) - .filter(filter) - .ignoreResiduals() - .planFiles(); - - try { - StructType partitionType = table.spec().partitionType(); - StructLikeMap> filesByPartition = - groupByPartition(partitionType, fileScanTasks); - return fileGroupsByPartition(filesByPartition); - } finally { - try { - fileScanTasks.close(); - } catch (IOException io) { - LOG.error("Cannot properly close file iterable while planning for rewrite", io); - } - } + @VisibleForTesting + RewriteFilePlan plan() { + return planner.plan(); } - private StructLikeMap> groupByPartition( - StructType partitionType, Iterable tasks) { - StructLikeMap> filesByPartition = StructLikeMap.create(partitionType); - StructLike emptyStruct = GenericRecord.create(partitionType); - - for (FileScanTask task : tasks) { - // If a task uses an incompatible partition spec the data inside could contain values - // which belong to multiple partitions in the current spec. Treating all such files as - // un-partitioned and grouping them together helps to minimize new files made. - StructLike taskPartition = - task.file().specId() == table.spec().specId() ? task.file().partition() : emptyStruct; - - List files = filesByPartition.get(taskPartition); - if (files == null) { - files = Lists.newArrayList(); - } - - files.add(task); - filesByPartition.put(taskPartition, files); - } - return filesByPartition; - } + @VisibleForTesting + void init(long startingSnapshotId) { - private StructLikeMap>> fileGroupsByPartition( - StructLikeMap> filesByPartition) { - return filesByPartition.transformValues(this::planFileGroups); - } + this.planner = + shufflingPlanner + ? new SparkShufflingDataRewritePlanner(table, filter, startingSnapshotId, caseSensitive) + : new RewriteFileGroupPlanner(table, filter, startingSnapshotId, caseSensitive); + + // Default to BinPack if no strategy selected + if (this.rewriter == null) { + this.rewriter = new SparkBinPackDataRewriteExecutor(spark(), table); + } - private List> planFileGroups(List tasks) { - return ImmutableList.copyOf(rewriter.planFileGroups(tasks)); + validateAndInitOptions(); } @VisibleForTesting - RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) { - String desc = jobDesc(fileGroup, ctx); + RewriteFileGroup rewriteFiles(RewriteFilePlan plan, RewriteFileGroup fileGroup) { + String desc = jobDesc(fileGroup, plan); Set addedFiles = withJobGroupInfo( - newJobGroupInfo("REWRITE-DATA-FILES", desc), - () -> rewriter.rewrite(fileGroup.fileScans())); + newJobGroupInfo("REWRITE-DATA-FILES", desc), () -> rewriter.rewrite(fileGroup)); fileGroup.setOutputFiles(addedFiles); LOG.info("Rewrite Files Ready to be Committed - {}", desc); @@ -279,16 +234,13 @@ RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { table, startingSnapshotId, useStartingSequenceNumber, commitSummary()); } - private Builder doExecute( - RewriteExecutionContext ctx, - Stream groupStream, - RewriteDataFilesCommitManager commitManager) { + private Builder doExecute(RewriteFilePlan plan, RewriteDataFilesCommitManager commitManager) { ExecutorService rewriteService = rewriteService(); ConcurrentLinkedQueue rewrittenGroups = Queues.newConcurrentLinkedQueue(); Tasks.Builder rewriteTaskBuilder = - Tasks.foreach(groupStream) + Tasks.foreach(plan.groups()) .executeWith(rewriteService) .stopOnFailure() .noRetry() @@ -299,10 +251,7 @@ private Builder doExecute( }); try { - rewriteTaskBuilder.run( - fileGroup -> { - rewrittenGroups.add(rewriteFiles(ctx, fileGroup)); - }); + rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteFiles(plan, fileGroup))); } catch (Exception e) { // At least one rewrite group failed, clean up all completed rewrites LOG.error( @@ -345,20 +294,18 @@ private Builder doExecute( } private Builder doExecuteWithPartialProgress( - RewriteExecutionContext ctx, - Stream groupStream, - RewriteDataFilesCommitManager commitManager) { + RewriteFilePlan plan, RewriteDataFilesCommitManager commitManager) { ExecutorService rewriteService = rewriteService(); // start commit service - int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING); + int groupsPerCommit = IntMath.divide(plan.totalGroupCount(), maxCommits, RoundingMode.CEILING); RewriteDataFilesCommitManager.CommitService commitService = commitManager.service(groupsPerCommit); commitService.start(); Collection rewriteFailures = new ConcurrentLinkedQueue<>(); // start rewrite tasks - Tasks.foreach(groupStream) + Tasks.foreach(plan.groups()) .suppressFailureWhenFinished() .executeWith(rewriteService) .noRetry() @@ -368,10 +315,10 @@ private Builder doExecuteWithPartialProgress( rewriteFailures.add( ImmutableRewriteDataFiles.FileGroupFailureResult.builder() .info(fileGroup.info()) - .dataFilesCount(fileGroup.numFiles()) + .dataFilesCount(fileGroup.numInputFiles()) .build()); }) - .run(fileGroup -> commitService.offer(rewriteFiles(ctx, fileGroup))); + .run(fileGroup -> commitService.offer(rewriteFiles(plan, fileGroup))); rewriteService.shutdown(); // stop commit service @@ -404,32 +351,6 @@ private Builder doExecuteWithPartialProgress( .rewriteFailures(rewriteFailures); } - Stream toGroupStream( - RewriteExecutionContext ctx, Map>> groupsByPartition) { - return groupsByPartition.entrySet().stream() - .filter(e -> !e.getValue().isEmpty()) - .flatMap( - e -> { - StructLike partition = e.getKey(); - List> scanGroups = e.getValue(); - return scanGroups.stream().map(tasks -> newRewriteGroup(ctx, partition, tasks)); - }) - .sorted(RewriteFileGroup.comparator(rewriteJobOrder)); - } - - private RewriteFileGroup newRewriteGroup( - RewriteExecutionContext ctx, StructLike partition, List tasks) { - int globalIndex = ctx.currentGlobalIndex(); - int partitionIndex = ctx.currentPartitionIndex(partition); - FileGroupInfo info = - ImmutableRewriteDataFiles.FileGroupInfo.builder() - .globalIndex(globalIndex) - .partitionIndex(partitionIndex) - .partition(partition) - .build(); - return new RewriteFileGroup(info, tasks); - } - private Iterable toRewriteResults(List commitResults) { return commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); } @@ -437,6 +358,7 @@ private Iterable toRewriteResults(List void validateAndInitOptions() { Set validOptions = Sets.newHashSet(rewriter.validOptions()); validOptions.addAll(VALID_OPTIONS); + validOptions.addAll(planner.validOptions()); Set invalidKeys = Sets.newHashSet(options().keySet()); invalidKeys.removeAll(validOptions); @@ -447,6 +369,7 @@ void validateAndInitOptions() { invalidKeys, rewriter.description()); + planner.init(options()); rewriter.init(options()); maxConcurrentFileGroupRewrites = @@ -474,10 +397,6 @@ void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); - rewriteJobOrder = - RewriteJobOrder.fromName( - PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); - Preconditions.checkArgument( maxConcurrentFileGroupRewrites >= 1, "Cannot set %s to %s, the value must be positive.", @@ -492,7 +411,7 @@ void validateAndInitOptions() { PARTIAL_PROGRESS_ENABLED); } - private String jobDesc(RewriteFileGroup group, RewriteExecutionContext ctx) { + private String jobDesc(RewriteFileGroup group, RewriteFilePlan plan) { StructLike partition = group.info().partition(); if (partition.size() > 0) { return String.format( @@ -500,10 +419,10 @@ private String jobDesc(RewriteFileGroup group, RewriteExecutionContext ctx) { group.rewrittenFiles().size(), rewriter.description(), group.info().globalIndex(), - ctx.totalGroupCount(), + plan.totalGroupCount(), partition, group.info().partitionIndex(), - ctx.groupsInPartition(partition), + plan.groupsInPartition(partition), table.name()); } else { return String.format( @@ -511,39 +430,8 @@ private String jobDesc(RewriteFileGroup group, RewriteExecutionContext ctx) { group.rewrittenFiles().size(), rewriter.description(), group.info().globalIndex(), - ctx.totalGroupCount(), + plan.totalGroupCount(), table.name()); } } - - @VisibleForTesting - static class RewriteExecutionContext { - private final StructLikeMap numGroupsByPartition; - private final int totalGroupCount; - private final Map partitionIndexMap; - private final AtomicInteger groupIndex; - - RewriteExecutionContext(StructLikeMap>> fileGroupsByPartition) { - this.numGroupsByPartition = fileGroupsByPartition.transformValues(List::size); - this.totalGroupCount = numGroupsByPartition.values().stream().reduce(Integer::sum).orElse(0); - this.partitionIndexMap = Maps.newConcurrentMap(); - this.groupIndex = new AtomicInteger(1); - } - - public int currentGlobalIndex() { - return groupIndex.getAndIncrement(); - } - - public int currentPartitionIndex(StructLike partition) { - return partitionIndexMap.merge(partition, 1, Integer::sum); - } - - public int groupsInPartition(StructLike partition) { - return numGroupsByPartition.get(partition); - } - - public int totalGroupCount() { - return totalGroupCount; - } - } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java index 2562c74eafcc..4fc7934f783b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java @@ -18,52 +18,37 @@ */ package org.apache.iceberg.spark.actions; -import java.io.IOException; import java.math.RoundingMode; 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.PositionDeletesTable.PositionDeletesBatchScan; -import org.apache.iceberg.RewriteJobOrder; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles; import org.apache.iceberg.actions.RewritePositionDeleteFiles; +import org.apache.iceberg.actions.RewritePositionDeletePlan; import org.apache.iceberg.actions.RewritePositionDeletesCommitManager; import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService; import org.apache.iceberg.actions.RewritePositionDeletesGroup; +import org.apache.iceberg.actions.RewritePositionDeletesGroupPlanner; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.io.CloseableIterable; 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.spark.SparkUtil; -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.slf4j.Logger; @@ -86,20 +71,20 @@ public class RewritePositionDeleteFilesSparkAction ImmutableRewritePositionDeleteFiles.Result.builder().build(); private final Table table; - private final SparkBinPackPositionDeletesRewriter rewriter; + private RewritePositionDeletesGroupPlanner planner; + private final SparkBinPackPositionDeletesRewriteExecutor rewriter; private Expression filter = Expressions.alwaysTrue(); private int maxConcurrentFileGroupRewrites; private int maxCommits; private boolean partialProgressEnabled; - private RewriteJobOrder rewriteJobOrder; private boolean caseSensitive; RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) { super(spark); this.table = table; - this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table); this.caseSensitive = SparkUtil.caseSensitive(spark); + this.rewriter = new SparkBinPackPositionDeletesRewriteExecutor(spark(), table); } @Override @@ -120,86 +105,31 @@ public RewritePositionDeleteFiles.Result execute() { return EMPTY_RESULT; } + this.planner = new RewritePositionDeletesGroupPlanner(table, filter, caseSensitive); + validateAndInitOptions(); - StructLikeMap>> fileGroupsByPartition = planFileGroups(); - RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition); + RewritePositionDeletePlan plan = planner.plan(); + rewriter.initPlan(plan); - if (ctx.totalGroupCount() == 0) { + if (plan.totalGroupCount() == 0) { LOG.info("Nothing found to rewrite in {}", table.name()); return EMPTY_RESULT; } - Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); - if (partialProgressEnabled) { - return doExecuteWithPartialProgress(ctx, groupStream, commitManager()); + return doExecuteWithPartialProgress(plan, commitManager()); } else { - return doExecute(ctx, groupStream, commitManager()); - } - } - - private StructLikeMap>> planFileGroups() { - Table deletesTable = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); - CloseableIterable fileTasks = planFiles(deletesTable); - - try { - StructType partitionType = Partitioning.partitionType(deletesTable); - StructLikeMap> fileTasksByPartition = - groupByPartition(partitionType, fileTasks); - return fileGroupsByPartition(fileTasksByPartition); - } finally { - try { - fileTasks.close(); - } catch (IOException io) { - LOG.error("Cannot properly close file iterable while planning for rewrite", io); - } + return doExecute(plan, commitManager()); } } - private CloseableIterable planFiles(Table deletesTable) { - PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); - return CloseableIterable.transform( - scan.baseTableFilter(filter).caseSensitive(caseSensitive).ignoreResiduals().planFiles(), - task -> (PositionDeletesScanTask) task); - } - - private StructLikeMap> groupByPartition( - StructType partitionType, Iterable tasks) { - StructLikeMap> filesByPartition = - StructLikeMap.create(partitionType); - - for (PositionDeletesScanTask task : tasks) { - StructLike coerced = coercePartition(task, partitionType); - - List partitionTasks = filesByPartition.get(coerced); - if (partitionTasks == null) { - partitionTasks = Lists.newArrayList(); - } - partitionTasks.add(task); - filesByPartition.put(coerced, partitionTasks); - } - - return filesByPartition; - } - - private StructLikeMap>> fileGroupsByPartition( - StructLikeMap> filesByPartition) { - return filesByPartition.transformValues(this::planFileGroups); - } - - private List> planFileGroups(List tasks) { - return ImmutableList.copyOf(rewriter.planFileGroups(tasks)); - } - private RewritePositionDeletesGroup rewriteDeleteFiles( - RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) { - String desc = jobDesc(fileGroup, ctx); + RewritePositionDeletePlan plan, RewritePositionDeletesGroup fileGroup) { + String desc = jobDesc(fileGroup, plan); Set addedFiles = withJobGroupInfo( - newJobGroupInfo("REWRITE-POSITION-DELETES", desc), - () -> rewriter.rewrite(fileGroup.tasks())); + newJobGroupInfo("REWRITE-POSITION-DELETES", desc), () -> rewriter.rewrite(fileGroup)); fileGroup.setOutputFiles(addedFiles); LOG.info("Rewrite position deletes ready to be committed - {}", desc); @@ -221,16 +151,14 @@ private RewritePositionDeletesCommitManager commitManager() { } private Result doExecute( - RewriteExecutionContext ctx, - Stream groupStream, - RewritePositionDeletesCommitManager commitManager) { + RewritePositionDeletePlan plan, RewritePositionDeletesCommitManager commitManager) { ExecutorService rewriteService = rewriteService(); ConcurrentLinkedQueue rewrittenGroups = Queues.newConcurrentLinkedQueue(); Tasks.Builder rewriteTaskBuilder = - Tasks.foreach(groupStream) + Tasks.foreach(plan.groups()) .executeWith(rewriteService) .stopOnFailure() .noRetry() @@ -242,7 +170,7 @@ private Result doExecute( exception)); try { - rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup))); + rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(plan, fileGroup))); } catch (Exception e) { // At least one rewrite group failed, clean up all completed rewrites LOG.error( @@ -288,25 +216,23 @@ private Result doExecute( } private Result doExecuteWithPartialProgress( - RewriteExecutionContext ctx, - Stream groupStream, - RewritePositionDeletesCommitManager commitManager) { + RewritePositionDeletePlan plan, RewritePositionDeletesCommitManager commitManager) { ExecutorService rewriteService = rewriteService(); // start commit service - int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING); + int groupsPerCommit = IntMath.divide(plan.totalGroupCount(), maxCommits, RoundingMode.CEILING); CommitService commitService = commitManager.service(groupsPerCommit); commitService.start(); // start rewrite tasks - Tasks.foreach(groupStream) + Tasks.foreach(plan.groups()) .suppressFailureWhenFinished() .executeWith(rewriteService) .noRetry() .onFailure( (fileGroup, exception) -> LOG.error("Failure during rewrite group {}", fileGroup.info(), exception)) - .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup))); + .run(fileGroup -> commitService.offer(rewriteDeleteFiles(plan, fileGroup))); rewriteService.shutdown(); // stop commit service @@ -330,36 +256,10 @@ private Result doExecuteWithPartialProgress( .build(); } - private Stream toGroupStream( - RewriteExecutionContext ctx, - Map>> groupsByPartition) { - return groupsByPartition.entrySet().stream() - .filter(e -> !e.getValue().isEmpty()) - .flatMap( - e -> { - StructLike partition = e.getKey(); - List> scanGroups = e.getValue(); - return scanGroups.stream().map(tasks -> newRewriteGroup(ctx, partition, tasks)); - }) - .sorted(RewritePositionDeletesGroup.comparator(rewriteJobOrder)); - } - - private RewritePositionDeletesGroup newRewriteGroup( - RewriteExecutionContext ctx, StructLike partition, List tasks) { - int globalIndex = ctx.currentGlobalIndex(); - int partitionIndex = ctx.currentPartitionIndex(partition); - FileGroupInfo info = - ImmutableRewritePositionDeleteFiles.FileGroupInfo.builder() - .globalIndex(globalIndex) - .partitionIndex(partitionIndex) - .partition(partition) - .build(); - return new RewritePositionDeletesGroup(info, tasks); - } - private void validateAndInitOptions() { Set validOptions = Sets.newHashSet(rewriter.validOptions()); validOptions.addAll(VALID_OPTIONS); + validOptions.addAll(planner.validOptions()); Set invalidKeys = Sets.newHashSet(options().keySet()); invalidKeys.removeAll(validOptions); @@ -370,6 +270,7 @@ private void validateAndInitOptions() { invalidKeys, rewriter.description()); + planner.init(options()); rewriter.init(options()); this.maxConcurrentFileGroupRewrites = @@ -386,10 +287,6 @@ private void validateAndInitOptions() { PropertyUtil.propertyAsBoolean( options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT); - this.rewriteJobOrder = - RewriteJobOrder.fromName( - PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT)); - Preconditions.checkArgument( maxConcurrentFileGroupRewrites >= 1, "Cannot set %s to %s, the value must be positive.", @@ -404,7 +301,7 @@ private void validateAndInitOptions() { PARTIAL_PROGRESS_ENABLED); } - private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContext ctx) { + private String jobDesc(RewritePositionDeletesGroup group, RewritePositionDeletePlan plan) { StructLike partition = group.info().partition(); if (partition.size() > 0) { return String.format( @@ -412,10 +309,10 @@ private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContex group.rewrittenDeleteFiles().size(), rewriter.description(), group.info().globalIndex(), - ctx.totalGroupCount(), + plan.totalGroupCount(), partition, group.info().partitionIndex(), - ctx.groupsInPartition(partition), + plan.groupsInPartition(partition), table.name()); } else { return String.format( @@ -423,43 +320,8 @@ private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContex group.rewrittenDeleteFiles().size(), rewriter.description(), group.info().globalIndex(), - ctx.totalGroupCount(), + plan.totalGroupCount(), table.name()); } } - - static class RewriteExecutionContext { - private final StructLikeMap numGroupsByPartition; - private final int totalGroupCount; - private final Map partitionIndexMap; - private final AtomicInteger groupIndex; - - RewriteExecutionContext( - StructLikeMap>> fileTasksByPartition) { - this.numGroupsByPartition = fileTasksByPartition.transformValues(List::size); - this.totalGroupCount = numGroupsByPartition.values().stream().reduce(Integer::sum).orElse(0); - this.partitionIndexMap = Maps.newConcurrentMap(); - this.groupIndex = new AtomicInteger(1); - } - - public int currentGlobalIndex() { - return groupIndex.getAndIncrement(); - } - - public int currentPartitionIndex(StructLike partition) { - return partitionIndexMap.merge(partition, 1, Integer::sum); - } - - public int groupsInPartition(StructLike partition) { - return numGroupsByPartition.get(partition); - } - - public int totalGroupCount() { - return totalGroupCount; - } - } - - private StructLike coercePartition(PositionDeletesScanTask task, StructType partitionType) { - return PartitionUtil.coercePartition(partitionType, task.spec(), task.partition()); - } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriteExecutor.java similarity index 88% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriteExecutor.java index d256bf2794e2..d1c70ee289c6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriteExecutor.java @@ -28,9 +28,9 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -class SparkBinPackDataRewriter extends SparkSizeBasedDataRewriter { +class SparkBinPackDataRewriteExecutor extends SparkSizeBasedDataRewriteExecutor { - SparkBinPackDataRewriter(SparkSession spark, Table table) { + SparkBinPackDataRewriteExecutor(SparkSession spark, Table table) { super(spark, table); } @@ -40,14 +40,15 @@ public String description() { } @Override - protected void doRewrite(String groupId, List group) { + protected void doRewrite( + String groupId, List group, long splitSize, int expectedOutputFiles) { // read the files packing them into splits of the required size Dataset scanDF = spark() .read() .format("iceberg") .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) - .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group))) + .option(SparkReadOptions.SPLIT_SIZE, splitSize) .option(SparkReadOptions.FILE_OPEN_COST, "0") .load(groupId); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriteExecutor.java similarity index 87% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriteExecutor.java index 5afd724aad88..cfc939cf0347 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriteExecutor.java @@ -34,7 +34,9 @@ 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.actions.RewritePositionDeleteFiles.FileGroupInfo; +import org.apache.iceberg.actions.RewritePositionDeletePlan; +import org.apache.iceberg.actions.RewritePositionDeletesGroup; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; @@ -51,7 +53,13 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.internal.SQLConf; -class SparkBinPackPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter { +class SparkBinPackPositionDeletesRewriteExecutor + extends SparkRewriteExecutor< + FileGroupInfo, + PositionDeletesScanTask, + DeleteFile, + RewritePositionDeletesGroup, + RewritePositionDeletePlan> { private final SparkSession spark; private final SparkTableCache tableCache = SparkTableCache.get(); @@ -59,7 +67,7 @@ class SparkBinPackPositionDeletesRewriter extends SizeBasedPositionDeletesRewrit private final PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); - SparkBinPackPositionDeletesRewriter(SparkSession spark, Table table) { + SparkBinPackPositionDeletesRewriteExecutor(SparkSession spark, Table table) { super(table); // Disable Adaptive Query Execution as this may change the output partitioning of our write this.spark = spark.cloneSession(); @@ -72,14 +80,14 @@ public String description() { } @Override - public Set rewrite(List group) { + public Set rewrite(RewritePositionDeletesGroup group) { String groupId = UUID.randomUUID().toString(); Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table(), POSITION_DELETES); try { tableCache.add(groupId, deletesTable); - taskSetManager.stageTasks(deletesTable, groupId, group); + taskSetManager.stageTasks(deletesTable, groupId, group.fileScans()); - doRewrite(groupId, group); + doRewrite(groupId, group.fileScans(), group.splitSize()); return coordinator.fetchNewFiles(deletesTable, groupId); } finally { @@ -89,7 +97,7 @@ public Set rewrite(List group) { } } - protected void doRewrite(String groupId, List group) { + protected void doRewrite(String groupId, List group, long splitSize) { // all position deletes are of the same partition, because they are in same file group Preconditions.checkArgument(!group.isEmpty(), "Empty group"); Types.StructType partitionType = group.get(0).spec().partitionType(); @@ -101,7 +109,7 @@ protected void doRewrite(String groupId, List group) { .read() .format("iceberg") .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) - .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group))) + .option(SparkReadOptions.SPLIT_SIZE, splitSize) .option(SparkReadOptions.FILE_OPEN_COST, "0") .load(groupId); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteExecutor.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteExecutor.java new file mode 100644 index 000000000000..99c2f88a9b0a --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteExecutor.java @@ -0,0 +1,74 @@ +/* + * 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.util.Map; +import java.util.Set; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.FileRewriteExecutor; +import org.apache.iceberg.actions.FileRewriteGroup; +import org.apache.iceberg.actions.FileRewritePlan; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; + +/** + * Common parent for data and positional delete rewrite executors. + * + * @param the Java type of the plan info + * @param the Java type of the tasks to read content files + * @param the Java type of the content files + * @param the Java type of the planned groups + * @param

the Java type of the plan to execute + */ +abstract class SparkRewriteExecutor< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends FileRewriteGroup, + P extends FileRewritePlan> + implements FileRewriteExecutor { + private final Table table; + private long writeMaxFileSize; + + SparkRewriteExecutor(Table table) { + this.table = table; + } + + Table table() { + return table; + } + + long writeMaxFileSize() { + return writeMaxFileSize; + } + + @Override + public void initPlan(P plan) { + this.writeMaxFileSize = plan.writeMaxFileSize(); + } + + @Override + public Set validOptions() { + return ImmutableSet.of(); + } + + @Override + public void init(Map options) {} +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriteExecutor.java similarity index 80% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriteExecutor.java index ce572c6486cc..87b9326e3b6c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewriteExecutor.java @@ -48,20 +48,7 @@ import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; import scala.Option; -abstract class SparkShufflingDataRewriter extends SparkSizeBasedDataRewriter { - - /** - * The number of shuffle partitions and consequently the number of output files created by the - * Spark sort is based on the size of the input data files used in this file rewriter. Due to - * compression, the disk file sizes may not accurately represent the size of files in the output. - * This parameter lets the user adjust the file size used for estimating actual output data size. - * A factor greater than 1.0 would generate more files than we would expect based on the on-disk - * file size. A value less than 1.0 would create fewer files than we would expect based on the - * on-disk size. - */ - public static final String COMPRESSION_FACTOR = "compression-factor"; - - public static final double COMPRESSION_FACTOR_DEFAULT = 1.0; +abstract class SparkShufflingDataRewriteExecutor extends SparkSizeBasedDataRewriteExecutor { /** * The number of shuffle partitions to use for each output file. By default, this file rewriter @@ -79,10 +66,9 @@ abstract class SparkShufflingDataRewriter extends SparkSizeBasedDataRewriter { public static final int SHUFFLE_PARTITIONS_PER_FILE_DEFAULT = 1; - private double compressionFactor; private int numShufflePartitionsPerFile; - protected SparkShufflingDataRewriter(SparkSession spark, Table table) { + protected SparkShufflingDataRewriteExecutor(SparkSession spark, Table table) { super(spark, table); } @@ -105,7 +91,6 @@ protected abstract Dataset sortedDF( public Set validOptions() { return ImmutableSet.builder() .addAll(super.validOptions()) - .add(COMPRESSION_FACTOR) .add(SHUFFLE_PARTITIONS_PER_FILE) .build(); } @@ -113,12 +98,12 @@ public Set validOptions() { @Override public void init(Map options) { super.init(options); - this.compressionFactor = compressionFactor(options); this.numShufflePartitionsPerFile = numShufflePartitionsPerFile(options); } @Override - public void doRewrite(String groupId, List group) { + public void doRewrite( + String groupId, List group, long splitSize, int expectedOutputFiles) { Dataset scanDF = spark() .read() @@ -126,7 +111,7 @@ public void doRewrite(String groupId, List group) { .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) .load(groupId); - Dataset sortedDF = sortedDF(scanDF, sortFunction(group)); + Dataset sortedDF = sortedDF(scanDF, sortFunction(group, expectedOutputFiles)); sortedDF .write() @@ -139,9 +124,10 @@ public void doRewrite(String groupId, List group) { .save(groupId); } - private Function, Dataset> sortFunction(List group) { + private Function, Dataset> sortFunction( + List group, int expectedOutputFiles) { SortOrder[] ordering = Spark3Util.toOrdering(outputSortOrder(group)); - int numShufflePartitions = numShufflePartitions(group); + int numShufflePartitions = Math.max(1, expectedOutputFiles * numShufflePartitionsPerFile); return (df) -> transformPlan(df, plan -> sortPlan(plan, ordering, numShufflePartitions)); } @@ -176,19 +162,6 @@ private org.apache.iceberg.SortOrder outputSortOrder(List group) { } } - private int numShufflePartitions(List group) { - int numOutputFiles = (int) numOutputFiles((long) (inputSize(group) * compressionFactor)); - return Math.max(1, numOutputFiles * numShufflePartitionsPerFile); - } - - private double compressionFactor(Map options) { - double value = - PropertyUtil.propertyAsDouble(options, COMPRESSION_FACTOR, COMPRESSION_FACTOR_DEFAULT); - Preconditions.checkArgument( - value > 0, "'%s' is set to %s but must be > 0", COMPRESSION_FACTOR, value); - return value; - } - private int numShufflePartitionsPerFile(Map options) { int value = PropertyUtil.propertyAsInt( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java new file mode 100644 index 000000000000..16410946bd7d --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java @@ -0,0 +1,85 @@ +/* + * 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.util.Map; +import java.util.Set; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteFileGroupPlanner; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.PropertyUtil; + +/** + * Extends the {@link RewriteFileGroupPlanner} with the possibility to set the expected compression + * factor. + */ +public class SparkShufflingDataRewritePlanner extends RewriteFileGroupPlanner { + /** + * The number of shuffle partitions and consequently the number of output files created by the + * Spark sort is based on the size of the input data files used in this file rewriter. Due to + * compression, the disk file sizes may not accurately represent the size of files in the output. + * This parameter lets the user adjust the file size used for estimating actual output data size. + * A factor greater than 1.0 would generate more files than we would expect based on the on-disk + * file size. A value less than 1.0 would create fewer files than we would expect based on the + * on-disk size. + */ + public static final String COMPRESSION_FACTOR = "compression-factor"; + + public static final double COMPRESSION_FACTOR_DEFAULT = 1.0; + + private double compressionFactor; + + public SparkShufflingDataRewritePlanner(Table table) { + super(table); + } + + public SparkShufflingDataRewritePlanner( + Table table, Expression filter, Long snapshotId, boolean caseSensitive) { + super(table, filter, snapshotId, caseSensitive); + } + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(COMPRESSION_FACTOR) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.compressionFactor = compressionFactor(options); + } + + @Override + protected int numOutputFiles(long inputSize) { + return Math.max(1, super.numOutputFiles((long) (inputSize * compressionFactor))); + } + + private double compressionFactor(Map options) { + double value = + PropertyUtil.propertyAsDouble(options, COMPRESSION_FACTOR, COMPRESSION_FACTOR_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", COMPRESSION_FACTOR, value); + return value; + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriteExecutor.java similarity index 63% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriteExecutor.java index ae0e0d20dd4e..800882cc31e5 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSizeBasedDataRewriteExecutor.java @@ -23,39 +23,54 @@ import java.util.UUID; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; -import org.apache.iceberg.actions.SizeBasedDataRewriter; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.actions.RewriteFilePlan; import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkTableCache; import org.apache.spark.sql.SparkSession; -abstract class SparkSizeBasedDataRewriter extends SizeBasedDataRewriter { +abstract class SparkSizeBasedDataRewriteExecutor + extends SparkRewriteExecutor< + FileGroupInfo, FileScanTask, DataFile, RewriteFileGroup, RewriteFilePlan> { private final SparkSession spark; private final SparkTableCache tableCache = SparkTableCache.get(); private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + private int outputSpecId; - SparkSizeBasedDataRewriter(SparkSession spark, Table table) { + SparkSizeBasedDataRewriteExecutor(SparkSession spark, Table table) { super(table); this.spark = spark; } - protected abstract void doRewrite(String groupId, List group); + protected abstract void doRewrite( + String groupId, List group, long splitSize, int expectedOutputFiles); protected SparkSession spark() { return spark; } + protected int outputSpecId() { + return outputSpecId; + } + + protected PartitionSpec outputSpec() { + return table().specs().get(outputSpecId); + } + @Override - public Set rewrite(List group) { + public Set rewrite(RewriteFileGroup group) { String groupId = UUID.randomUUID().toString(); try { tableCache.add(groupId, table()); - taskSetManager.stageTasks(table(), groupId, group); + taskSetManager.stageTasks(table(), groupId, group.fileScans()); - doRewrite(groupId, group); + doRewrite(groupId, group.fileScans(), group.splitSize(), group.expectedOutputFiles()); return coordinator.fetchNewFiles(table(), groupId); } finally { @@ -64,4 +79,10 @@ public Set rewrite(List group) { coordinator.clearRewrite(table(), groupId); } } + + @Override + public void initPlan(RewriteFilePlan plan) { + super.initPlan(plan); + this.outputSpecId = plan.outputSpecId(); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriteExecutor.java similarity index 89% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriteExecutor.java index 1f70d4d7ca9d..a1d4c57894cc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortDataRewriteExecutor.java @@ -26,11 +26,11 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -class SparkSortDataRewriter extends SparkShufflingDataRewriter { +class SparkSortDataRewriteExecutor extends SparkShufflingDataRewriteExecutor { private final SortOrder sortOrder; - SparkSortDataRewriter(SparkSession spark, Table table) { + SparkSortDataRewriteExecutor(SparkSession spark, Table table) { super(spark, table); Preconditions.checkArgument( table.sortOrder().isSorted(), @@ -39,7 +39,7 @@ class SparkSortDataRewriter extends SparkShufflingDataRewriter { this.sortOrder = table.sortOrder(); } - SparkSortDataRewriter(SparkSession spark, Table table, SortOrder sortOrder) { + SparkSortDataRewriteExecutor(SparkSession spark, Table table, SortOrder sortOrder) { super(spark, table); Preconditions.checkArgument( sortOrder != null && sortOrder.isSorted(), diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriteExecutor.java similarity index 97% rename from spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriter.java rename to spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriteExecutor.java index cc4fb78ebd18..d4dc5affb4b6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderDataRewriteExecutor.java @@ -44,9 +44,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class SparkZOrderDataRewriter extends SparkShufflingDataRewriter { +class SparkZOrderDataRewriteExecutor extends SparkShufflingDataRewriteExecutor { - private static final Logger LOG = LoggerFactory.getLogger(SparkZOrderDataRewriter.class); + private static final Logger LOG = LoggerFactory.getLogger(SparkZOrderDataRewriteExecutor.class); private static final String Z_COLUMN = "ICEZVALUE"; private static final Schema Z_SCHEMA = @@ -78,7 +78,7 @@ class SparkZOrderDataRewriter extends SparkShufflingDataRewriter { private int maxOutputSize; private int varLengthContribution; - SparkZOrderDataRewriter(SparkSession spark, Table table, List zOrderColNames) { + SparkZOrderDataRewriteExecutor(SparkSession spark, Table table, List zOrderColNames) { super(spark, table); this.zOrderColNames = validZOrderColNames(spark, table, zOrderColNames); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 38c4d32a90d2..d25710e7cd13 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -76,8 +76,9 @@ import org.apache.iceberg.actions.RewriteDataFiles.Result; import org.apache.iceberg.actions.RewriteDataFilesCommitManager; import org.apache.iceberg.actions.RewriteFileGroup; -import org.apache.iceberg.actions.SizeBasedDataRewriter; -import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.actions.RewriteFileGroupPlanner; +import org.apache.iceberg.actions.RewriteFilePlan; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -108,7 +109,6 @@ import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.TestBase; -import org.apache.iceberg.spark.actions.RewriteDataFilesSparkAction.RewriteExecutionContext; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Comparators; @@ -117,7 +117,6 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -170,7 +169,9 @@ public void setupTableLocation() throws Exception { private RewriteDataFilesSparkAction basicRewrite(Table table) { // Always compact regardless of input files table.refresh(); - return actions().rewriteDataFiles(table).option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1"); + return actions() + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1"); } @TestTemplate @@ -291,9 +292,9 @@ public void testBinPackAfterPartitionChange() { RewriteDataFiles.Result result = basicRewrite(table) - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") .option( - SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table) + 1000)) .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, @@ -358,11 +359,12 @@ public void testBinPackWithDeletes() throws IOException { actions() .rewriteDataFiles(table) // do not include any file based on bin pack file size configs - .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "0") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) - .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) // set DELETE_FILE_THRESHOLD to 1 since DVs only produce one delete file per data file - .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "1") + .option(RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, "1") .execute(); assertThat(result.rewrittenDataFilesCount()) .as("Action should rewrite 5 data files") @@ -373,10 +375,11 @@ public void testBinPackWithDeletes() throws IOException { actions() .rewriteDataFiles(table) // do not include any file based on bin pack file size configs - .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "0") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) - .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) - .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "2") + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + .option(RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, "2") .execute(); assertThat(result.rewrittenDataFilesCount()) .as("Action should rewrite 2 data files") @@ -434,7 +437,7 @@ public void testRemoveDangledEqualityDeletesPartitionEvolution() { RewriteDataFiles.Result result = basicRewrite(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .filter(Expressions.equal("c1", 1)) .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") .execute(); @@ -494,7 +497,7 @@ public void testRemoveDangledPositionDeletesPartitionEvolution() throws IOExcept actions() .rewriteDataFiles(table) .filter(Expressions.equal("c1", 1)) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") .execute(); @@ -540,7 +543,7 @@ public void testBinPackWithDeleteAllData() throws IOException { Result result = actions() .rewriteDataFiles(table) - .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "1") + .option(RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, "1") .execute(); assertThat(result.rewrittenDataFilesCount()).as("Action should rewrite 1 data files").isOne(); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); @@ -691,7 +694,9 @@ public void testBinPackSplitLargeFile() { Result result = basicRewrite(table) .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize)) - .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Long.toString(targetSize * 2 - 2000)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + Long.toString(targetSize * 2 - 2000)) .execute(); assertThat(result.rewrittenDataFilesCount()).as("Action should delete 1 data files").isOne(); @@ -722,8 +727,12 @@ public void testBinPackCombineMixedFiles() { Result result = basicRewrite(table) .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize + 1000)) - .option(SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 80000)) - .option(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 1000)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + Integer.toString(targetSize + 80000)) + .option( + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, + Integer.toString(targetSize - 1000)) .execute(); assertThat(result.rewrittenDataFilesCount()) @@ -754,10 +763,10 @@ public void testBinPackCombineMediumFiles() { basicRewrite(table) .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize)) .option( - SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, Integer.toString((int) (targetSize * 1.8))) .option( - SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100)) // All files too small .execute(); @@ -817,7 +826,7 @@ public void testMultipleGroups() { basicRewrite(table) .option( RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") .execute(); assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); @@ -879,7 +888,7 @@ public void testSingleCommitWithRewriteFailure() { GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); doThrow(new RuntimeException("Rewrite Failed")) .when(spyRewrite) - .rewriteFiles(any(), argThat(failGroup)); + .rewriteFiles(any(RewriteFilePlan.class), argThat(failGroup)); assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) @@ -982,7 +991,7 @@ public void testParallelSingleCommitWithRewriteFailure() { GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); doThrow(new CommitFailedException("Rewrite Failed")) .when(spyRewrite) - .rewriteFiles(any(), argThat(failGroup)); + .rewriteFiles(any(RewriteFilePlan.class), argThat(failGroup)); assertThatThrownBy(spyRewrite::execute) .isInstanceOf(CommitFailedException.class) @@ -1019,7 +1028,7 @@ public void testPartialProgressWithRewriteFailure() { GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); doThrow(new RuntimeException("Rewrite Failed")) .when(spyRewrite) - .rewriteFiles(any(), argThat(failGroup)); + .rewriteFiles(any(RewriteFilePlan.class), argThat(failGroup)); RewriteDataFiles.Result result = spyRewrite.execute(); @@ -1062,7 +1071,7 @@ public void testParallelPartialProgressWithRewriteFailure() { GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); doThrow(new RuntimeException("Rewrite Failed")) .when(spyRewrite) - .rewriteFiles(any(), argThat(failGroup)); + .rewriteFiles(any(RewriteFilePlan.class), argThat(failGroup)); RewriteDataFiles.Result result = spyRewrite.execute(); @@ -1150,7 +1159,7 @@ public void testParallelPartialProgressWithMaxFailedCommits() { GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); doThrow(new RuntimeException("Rewrite Failed")) .when(spyRewrite) - .rewriteFiles(any(), argThat(failGroup)); + .rewriteFiles(any(RewriteFilePlan.class), argThat(failGroup)); assertThatThrownBy(() -> spyRewrite.execute()) .isInstanceOf(RuntimeException.class) @@ -1208,7 +1217,7 @@ public void testInvalidOptions() { () -> basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) - .option(SparkShufflingDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, "5") + .option(SparkShufflingDataRewriteExecutor.SHUFFLE_PARTITIONS_PER_FILE, "5") .execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("requires enabling Iceberg Spark session extensions"); @@ -1229,7 +1238,7 @@ public void testSortMultipleGroups() { RewriteDataFiles.Result result = basicRewrite(table) .sort() - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option( RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) .execute(); @@ -1259,8 +1268,8 @@ public void testSimpleSort() { RewriteDataFiles.Result result = basicRewrite(table) .sort() - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); @@ -1293,8 +1302,8 @@ public void testSortAfterPartitionChange() { RewriteDataFiles.Result result = basicRewrite(table) .sort() - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); @@ -1327,7 +1336,7 @@ public void testSortCustomSortOrder() { RewriteDataFiles.Result result = basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) .execute(); @@ -1365,7 +1374,7 @@ public void testSortCustomSortOrderRequiresRepartition() { RewriteDataFiles.Result result = basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table) / partitions)) @@ -1399,13 +1408,13 @@ public void testAutoSortShuffleOutput() { basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) .option( - SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, Integer.toString((averageFileSize(table) / 2) + 2)) // Divide files in 2 .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table) / 2)) - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") .execute(); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); @@ -1479,13 +1488,13 @@ public void testZOrderSort() { basicRewrite(table) .zOrder("c2", "c3") .option( - SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, Integer.toString((averageFileSize(table) / 2) + 2)) // Divide files in 2 .option( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table) / 2)) - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") .execute(); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); @@ -1541,8 +1550,8 @@ public void testZOrderAllTypesSort() { "stringCol", "binaryCol", "booleanCol") - .option(SizeBasedFileRewriter.MIN_INPUT_FILES, "1") - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); @@ -1657,7 +1666,7 @@ public void testRewriteJobOrderFilesAsc() { RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = toGroupStream(table, basicRewrite) - .mapToLong(RewriteFileGroup::numFiles) + .mapToLong(RewriteFileGroup::numInputFiles) .boxed() .collect(Collectors.toList()); @@ -1667,7 +1676,7 @@ public void testRewriteJobOrderFilesAsc() { .binPack(); List actual = toGroupStream(table, jobOrderRewrite) - .mapToLong(RewriteFileGroup::numFiles) + .mapToLong(RewriteFileGroup::numInputFiles) .boxed() .collect(Collectors.toList()); @@ -1689,7 +1698,7 @@ public void testRewriteJobOrderFilesDesc() { RewriteDataFilesSparkAction basicRewrite = basicRewrite(table).binPack(); List expected = toGroupStream(table, basicRewrite) - .mapToLong(RewriteFileGroup::numFiles) + .mapToLong(RewriteFileGroup::numInputFiles) .boxed() .collect(Collectors.toList()); @@ -1699,7 +1708,7 @@ public void testRewriteJobOrderFilesDesc() { .binPack(); List actual = toGroupStream(table, jobOrderRewrite) - .mapToLong(RewriteFileGroup::numFiles) + .mapToLong(RewriteFileGroup::numInputFiles) .boxed() .collect(Collectors.toList()); @@ -1739,7 +1748,7 @@ public void testBinPackRewriterWithSpecificUnparitionedOutputSpec() { RewriteDataFiles.Result result = basicRewrite(table) .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .binPack() .execute(); @@ -1762,7 +1771,7 @@ public void testBinPackRewriterWithSpecificOutputSpec() { RewriteDataFiles.Result result = basicRewrite(table) .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .binPack() .execute(); @@ -1801,7 +1810,7 @@ public void testSortRewriterWithSpecificOutputSpecId() { RewriteDataFiles.Result result = basicRewrite(table) .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .sort(SortOrder.builderFor(table.schema()).asc("c2").asc("c3").build()) .execute(); @@ -1824,7 +1833,7 @@ public void testZOrderRewriteWithSpecificOutputSpecId() { RewriteDataFiles.Result result = basicRewrite(table) .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .zOrder("c2", "c3") .execute(); @@ -1851,12 +1860,9 @@ protected List currentDataFiles(Table table) { } private Stream toGroupStream(Table table, RewriteDataFilesSparkAction rewrite) { - rewrite.validateAndInitOptions(); - StructLikeMap>> fileGroupsByPartition = - rewrite.planFileGroups(table.currentSnapshot().snapshotId()); + rewrite.init(table.currentSnapshot().snapshotId()); - return rewrite.toGroupStream( - new RewriteExecutionContext(fileGroupsByPartition), fileGroupsByPartition); + return rewrite.plan().groups(); } protected List currentData() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 12b104fca27c..0fa8ccee9903 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -57,7 +57,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; -import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.FileHelpers; import org.apache.iceberg.deletes.DeleteGranularity; @@ -166,7 +166,7 @@ private void checkDeleteGranularity(DeleteGranularity deleteGranularity) throws Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); int expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? 2 : 1; @@ -191,7 +191,7 @@ public void testUnpartitioned() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); assertThat(newDeleteFiles).as("Expected 1 new delete file").hasSize(1); @@ -225,8 +225,10 @@ public void testRewriteAll() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") - .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) .execute(); List newDeleteFiles = deleteFiles(table); @@ -270,8 +272,10 @@ public void testRewriteFilter() throws Exception { SparkActions.get(spark) .rewritePositionDeletes(table) .filter(filter) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") - .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); @@ -322,8 +326,8 @@ public void testRewriteToSmallerTarget() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") - .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, String.valueOf(avgSize / 2)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, String.valueOf(avgSize / 2)) .execute(); List newDeleteFiles = deleteFiles(table); assertThat(newDeleteFiles).as("Should have 8 new delete files").hasSize(8); @@ -362,13 +366,13 @@ public void testRemoveDanglingDeletes() throws Exception { SparkActions.get(spark) .rewriteDataFiles(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); assertThat(newDeleteFiles).as("Should have 0 new delete files").hasSize(0); @@ -404,13 +408,13 @@ public void testSomePartitionsDanglingDeletes() throws Exception { SparkActions.get(spark) .rewriteDataFiles(table) .filter(filter) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); assertThat(newDeleteFiles).as("Should have 2 new delete files").hasSize(2); @@ -456,7 +460,7 @@ public void testRewriteFilterRemoveDangling() throws Exception { SparkActions.get(spark) .rewriteDataFiles(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); Expression filter = Expressions.or(Expressions.equal("c1", 0), Expressions.equal("c1", 1)); @@ -464,8 +468,10 @@ public void testRewriteFilterRemoveDangling() throws Exception { SparkActions.get(spark) .rewritePositionDeletes(table) .filter(filter) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") - .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) .execute(); List newDeleteFiles = except(deleteFiles(table), deleteFiles); @@ -517,7 +523,7 @@ public void testPartitionEvolutionAdd() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List rewrittenDeleteFiles = @@ -568,7 +574,7 @@ public void testPartitionEvolutionRemove() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List newDeleteFiles = deleteFiles(table); assertThat(newDeleteFiles).as("Should have 3 new delete files").hasSize(3); @@ -615,7 +621,7 @@ public void testSchemaEvolution() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); List rewrittenDeleteFiles = @@ -646,7 +652,7 @@ public void testSnapshotProperty() throws Exception { SparkActions.get(spark) .rewritePositionDeletes(table) .snapshotProperty("key", "value") - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); assertThat(table.currentSnapshot().summary()) .containsAllEntriesOf(ImmutableMap.of("key", "value")); @@ -711,8 +717,10 @@ public void testRewriteManyColumns() throws Exception { Result result = SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") - .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) .execute(); List newDeleteFiles = deleteFiles(table); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteExecutor.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteExecutor.java new file mode 100644 index 000000000000..32d1816e56e4 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteExecutor.java @@ -0,0 +1,150 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StringType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +class TestSparkFileRewriteExecutor extends TestBase { + + private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); + private static final Schema SCHEMA = + new Schema( + NestedField.required(1, "id", IntegerType.get()), + NestedField.required(2, "dep", StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dep").build(); + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @AfterEach + public void removeTable() { + catalog.dropTable(TABLE_IDENT); + } + + @Test + void testInvalidConstructorUsagesSortData() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + + assertThatThrownBy(() -> new SparkSortDataRewriteExecutor(spark, table)) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("is unsorted and no sort order is provided"); + + assertThatThrownBy(() -> new SparkSortDataRewriteExecutor(spark, table, null)) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("the provided sort order is null or empty"); + + assertThatThrownBy(() -> new SparkSortDataRewriteExecutor(spark, table, SortOrder.unsorted())) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("the provided sort order is null or empty"); + } + + @Test + void testInvalidConstructorUsagesZOrderData() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); + + assertThatThrownBy(() -> new SparkZOrderDataRewriteExecutor(spark, table, null)) + .hasMessageContaining("Cannot ZOrder when no columns are specified"); + + assertThatThrownBy(() -> new SparkZOrderDataRewriteExecutor(spark, table, ImmutableList.of())) + .hasMessageContaining("Cannot ZOrder when no columns are specified"); + + assertThatThrownBy( + () -> new SparkZOrderDataRewriteExecutor(spark, table, ImmutableList.of("dep"))) + .hasMessageContaining("Cannot ZOrder") + .hasMessageContaining("all columns provided were identity partition columns"); + + assertThatThrownBy( + () -> new SparkZOrderDataRewriteExecutor(spark, table, ImmutableList.of("DeP"))) + .hasMessageContaining("Cannot ZOrder") + .hasMessageContaining("all columns provided were identity partition columns"); + } + + @Test + void testBinPackDataValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkBinPackDataRewriteExecutor rewriter = new SparkBinPackDataRewriteExecutor(spark, table); + + assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo(ImmutableSet.of()); + } + + @Test + void testSortDataValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkSortDataRewriteExecutor rewriter = + new SparkSortDataRewriteExecutor(spark, table, SORT_ORDER); + + assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo(ImmutableSet.of(SparkSortDataRewriteExecutor.SHUFFLE_PARTITIONS_PER_FILE)); + } + + @Test + void testZOrderDataValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + ImmutableList zOrderCols = ImmutableList.of("id"); + SparkZOrderDataRewriteExecutor rewriter = + new SparkZOrderDataRewriteExecutor(spark, table, zOrderCols); + + assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkZOrderDataRewriteExecutor.SHUFFLE_PARTITIONS_PER_FILE, + SparkZOrderDataRewriteExecutor.MAX_OUTPUT_SIZE, + SparkZOrderDataRewriteExecutor.VAR_LENGTH_CONTRIBUTION)); + } + + @Test + void testInvalidValuesForZOrderDataOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + ImmutableList zOrderCols = ImmutableList.of("id"); + SparkZOrderDataRewriteExecutor rewriter = + new SparkZOrderDataRewriteExecutor(spark, table, zOrderCols); + + Map invalidMaxOutputOptions = + ImmutableMap.of(SparkZOrderDataRewriteExecutor.MAX_OUTPUT_SIZE, "0"); + assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) + .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") + .hasMessageContaining("'max-output-size' was set to 0"); + + Map invalidVarLengthContributionOptions = + ImmutableMap.of(SparkZOrderDataRewriteExecutor.VAR_LENGTH_CONTRIBUTION, "0"); + assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) + .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") + .hasMessageContaining("'var-length-contribution' was set to 0"); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewritePlanner.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewritePlanner.java new file mode 100644 index 000000000000..3426a6a71adb --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewritePlanner.java @@ -0,0 +1,104 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteFileGroupPlanner; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StringType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +class TestSparkFileRewritePlanner extends TestBase { + + private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); + private static final Schema SCHEMA = + new Schema( + NestedField.required(1, "id", IntegerType.get()), + NestedField.required(2, "dep", StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dep").build(); + + @AfterEach + public void removeTable() { + catalog.dropTable(TABLE_IDENT); + } + + @Test + void testRewriteFileGroupPlannerValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + RewriteFileGroupPlanner planner = new RewriteFileGroupPlanner(table); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + RewriteFileGroupPlanner.TARGET_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MAX_FILE_SIZE_BYTES, + RewriteFileGroupPlanner.MIN_INPUT_FILES, + RewriteFileGroupPlanner.REWRITE_ALL, + RewriteFileGroupPlanner.MAX_FILE_GROUP_SIZE_BYTES, + RewriteFileGroupPlanner.DELETE_FILE_THRESHOLD, + RewriteDataFiles.REWRITE_JOB_ORDER)); + } + + @Test + void testSparkShufflingDataRewritePlannerValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkShufflingDataRewritePlanner planner = new SparkShufflingDataRewritePlanner(table); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkShufflingDataRewritePlanner.COMPRESSION_FACTOR, + SparkShufflingDataRewritePlanner.TARGET_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MIN_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MAX_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MIN_INPUT_FILES, + SparkShufflingDataRewritePlanner.REWRITE_ALL, + SparkShufflingDataRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES, + SparkShufflingDataRewritePlanner.DELETE_FILE_THRESHOLD, + RewriteDataFiles.REWRITE_JOB_ORDER)); + } + + @Test + void testInvalidValuesSparkShufflingDataRewritePlannerOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkShufflingDataRewritePlanner planner = new SparkShufflingDataRewritePlanner(table); + + Map invalidCompressionFactorOptions = + ImmutableMap.of(SparkShufflingDataRewritePlanner.COMPRESSION_FACTOR, "0"); + assertThatThrownBy(() -> planner.init(invalidCompressionFactorOptions)) + .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java deleted file mode 100644 index e223d2e16411..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ /dev/null @@ -1,395 +0,0 @@ -/* - * 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.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import java.util.Map; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MockFileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.SizeBasedDataRewriter; -import org.apache.iceberg.actions.SizeBasedFileRewriter; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.spark.TestBase; -import org.apache.iceberg.types.Types.IntegerType; -import org.apache.iceberg.types.Types.NestedField; -import org.apache.iceberg.types.Types.StringType; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Test; - -public class TestSparkFileRewriter extends TestBase { - - private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); - private static final Schema SCHEMA = - new Schema( - NestedField.required(1, "id", IntegerType.get()), - NestedField.required(2, "dep", StringType.get())); - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("dep").build(); - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - - @AfterEach - public void removeTable() { - catalog.dropTable(TABLE_IDENT); - } - - @Test - public void testBinPackDataSelectFiles() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkBinPackDataRewriter rewriter = new SparkBinPackDataRewriter(spark, table); - - checkDataFileSizeFiltering(rewriter); - checkDataFilesDeleteThreshold(rewriter); - checkDataFileGroupWithEnoughFiles(rewriter); - checkDataFileGroupWithEnoughData(rewriter); - checkDataFileGroupWithTooMuchData(rewriter); - } - - @Test - public void testSortDataSelectFiles() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkSortDataRewriter rewriter = new SparkSortDataRewriter(spark, table, SORT_ORDER); - - checkDataFileSizeFiltering(rewriter); - checkDataFilesDeleteThreshold(rewriter); - checkDataFileGroupWithEnoughFiles(rewriter); - checkDataFileGroupWithEnoughData(rewriter); - checkDataFileGroupWithTooMuchData(rewriter); - } - - @Test - public void testZOrderDataSelectFiles() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - ImmutableList zOrderCols = ImmutableList.of("id"); - SparkZOrderDataRewriter rewriter = new SparkZOrderDataRewriter(spark, table, zOrderCols); - - checkDataFileSizeFiltering(rewriter); - checkDataFilesDeleteThreshold(rewriter); - checkDataFileGroupWithEnoughFiles(rewriter); - checkDataFileGroupWithEnoughData(rewriter); - checkDataFileGroupWithTooMuchData(rewriter); - } - - private void checkDataFileSizeFiltering(SizeBasedDataRewriter rewriter) { - FileScanTask tooSmallTask = new MockFileScanTask(100L); - FileScanTask optimal = new MockFileScanTask(450); - FileScanTask tooBigTask = new MockFileScanTask(1000L); - List tasks = ImmutableList.of(tooSmallTask, optimal, tooBigTask); - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, "250", - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, "500", - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, "750", - SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); - rewriter.init(options); - - Iterable> groups = rewriter.planFileGroups(tasks); - assertThat(groups).as("Must have 1 group").hasSize(1); - List group = Iterables.getOnlyElement(groups); - assertThat(group).as("Must rewrite 2 files").hasSize(2); - } - - private void checkDataFilesDeleteThreshold(SizeBasedDataRewriter rewriter) { - FileScanTask tooManyDeletesTask = MockFileScanTask.mockTaskWithDeletes(1000L, 3); - FileScanTask optimalTask = MockFileScanTask.mockTaskWithDeletes(1000L, 1); - List tasks = ImmutableList.of(tooManyDeletesTask, optimalTask); - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, "1", - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, "2000", - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, "5000", - SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "2"); - rewriter.init(options); - - Iterable> groups = rewriter.planFileGroups(tasks); - assertThat(groups).as("Must have 1 group").hasSize(1); - List group = Iterables.getOnlyElement(groups); - assertThat(group).as("Must rewrite 1 file").hasSize(1); - } - - private void checkDataFileGroupWithEnoughFiles(SizeBasedDataRewriter rewriter) { - List tasks = - ImmutableList.of( - new MockFileScanTask(100L), - new MockFileScanTask(100L), - new MockFileScanTask(100L), - new MockFileScanTask(100L)); - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_INPUT_FILES, "3", - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, "150", - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, "1000", - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, "5000", - SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); - rewriter.init(options); - - Iterable> groups = rewriter.planFileGroups(tasks); - assertThat(groups).as("Must have 1 group").hasSize(1); - List group = Iterables.getOnlyElement(groups); - assertThat(group).as("Must rewrite 4 files").hasSize(4); - } - - private void checkDataFileGroupWithEnoughData(SizeBasedDataRewriter rewriter) { - List tasks = - ImmutableList.of( - new MockFileScanTask(100L), new MockFileScanTask(100L), new MockFileScanTask(100L)); - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_INPUT_FILES, "5", - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, "200", - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, "250", - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, "500", - SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); - rewriter.init(options); - - Iterable> groups = rewriter.planFileGroups(tasks); - assertThat(groups).as("Must have 1 group").hasSize(1); - List group = Iterables.getOnlyElement(groups); - assertThat(group).as("Must rewrite 3 files").hasSize(3); - } - - private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { - List tasks = ImmutableList.of(new MockFileScanTask(2000L)); - - Map options = - ImmutableMap.of( - SizeBasedDataRewriter.MIN_INPUT_FILES, "5", - SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, "200", - SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, "250", - SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, "500", - SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, String.valueOf(Integer.MAX_VALUE)); - rewriter.init(options); - - Iterable> groups = rewriter.planFileGroups(tasks); - assertThat(groups).as("Must have 1 group").hasSize(1); - List group = Iterables.getOnlyElement(groups); - assertThat(group).as("Must rewrite big file").hasSize(1); - } - - @Test - public void testInvalidConstructorUsagesSortData() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - - assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) - .hasMessageContaining("Cannot sort data without a valid sort order") - .hasMessageContaining("is unsorted and no sort order is provided"); - - assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) - .hasMessageContaining("Cannot sort data without a valid sort order") - .hasMessageContaining("the provided sort order is null or empty"); - - assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) - .hasMessageContaining("Cannot sort data without a valid sort order") - .hasMessageContaining("the provided sort order is null or empty"); - } - - @Test - public void testInvalidConstructorUsagesZOrderData() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); - - assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) - .hasMessageContaining("Cannot ZOrder when no columns are specified"); - - assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) - .hasMessageContaining("Cannot ZOrder when no columns are specified"); - - assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) - .hasMessageContaining("Cannot ZOrder") - .hasMessageContaining("all columns provided were identity partition columns"); - - assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) - .hasMessageContaining("Cannot ZOrder") - .hasMessageContaining("all columns provided were identity partition columns"); - } - - @Test - public void testBinPackDataValidOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkBinPackDataRewriter rewriter = new SparkBinPackDataRewriter(spark, table); - - assertThat(rewriter.validOptions()) - .as("Rewriter must report all supported options") - .isEqualTo( - ImmutableSet.of( - SparkBinPackDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MIN_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MAX_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MIN_INPUT_FILES, - SparkBinPackDataRewriter.REWRITE_ALL, - SparkBinPackDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkBinPackDataRewriter.DELETE_FILE_THRESHOLD)); - } - - @Test - public void testSortDataValidOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkSortDataRewriter rewriter = new SparkSortDataRewriter(spark, table, SORT_ORDER); - - assertThat(rewriter.validOptions()) - .as("Rewriter must report all supported options") - .isEqualTo( - ImmutableSet.of( - SparkSortDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, - SparkSortDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkSortDataRewriter.MIN_FILE_SIZE_BYTES, - SparkSortDataRewriter.MAX_FILE_SIZE_BYTES, - SparkSortDataRewriter.MIN_INPUT_FILES, - SparkSortDataRewriter.REWRITE_ALL, - SparkSortDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkSortDataRewriter.DELETE_FILE_THRESHOLD, - SparkSortDataRewriter.COMPRESSION_FACTOR)); - } - - @Test - public void testZOrderDataValidOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - ImmutableList zOrderCols = ImmutableList.of("id"); - SparkZOrderDataRewriter rewriter = new SparkZOrderDataRewriter(spark, table, zOrderCols); - - assertThat(rewriter.validOptions()) - .as("Rewriter must report all supported options") - .isEqualTo( - ImmutableSet.of( - SparkZOrderDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, - SparkZOrderDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MIN_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MAX_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MIN_INPUT_FILES, - SparkZOrderDataRewriter.REWRITE_ALL, - SparkZOrderDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkZOrderDataRewriter.DELETE_FILE_THRESHOLD, - SparkZOrderDataRewriter.COMPRESSION_FACTOR, - SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, - SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION)); - } - - @Test - public void testInvalidValuesForBinPackDataOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkBinPackDataRewriter rewriter = new SparkBinPackDataRewriter(spark, table); - - validateSizeBasedRewriterOptions(rewriter); - - Map invalidDeleteThresholdOptions = - ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) - .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); - } - - @Test - public void testInvalidValuesForSortDataOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - SparkSortDataRewriter rewriter = new SparkSortDataRewriter(spark, table, SORT_ORDER); - - validateSizeBasedRewriterOptions(rewriter); - - Map invalidDeleteThresholdOptions = - ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) - .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); - - Map invalidCompressionFactorOptions = - ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) - .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); - } - - @Test - public void testInvalidValuesForZOrderDataOptions() { - Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - ImmutableList zOrderCols = ImmutableList.of("id"); - SparkZOrderDataRewriter rewriter = new SparkZOrderDataRewriter(spark, table, zOrderCols); - - validateSizeBasedRewriterOptions(rewriter); - - Map invalidDeleteThresholdOptions = - ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) - .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); - - Map invalidCompressionFactorOptions = - ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) - .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); - - Map invalidMaxOutputOptions = - ImmutableMap.of(SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, "0"); - assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) - .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") - .hasMessageContaining("'max-output-size' was set to 0"); - - Map invalidVarLengthContributionOptions = - ImmutableMap.of(SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION, "0"); - assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) - .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") - .hasMessageContaining("'var-length-contribution' was set to 0"); - } - - private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewriter) { - Map invalidTargetSizeOptions = - ImmutableMap.of(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "0"); - assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) - .hasMessageContaining("'target-file-size-bytes' is set to 0 but must be > 0"); - - Map invalidMinSizeOptions = - ImmutableMap.of(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "-1"); - assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) - .hasMessageContaining("'min-file-size-bytes' is set to -1 but must be >= 0"); - - Map invalidTargetMinSizeOptions = - ImmutableMap.of( - SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "3", - SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "5"); - assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) - .hasMessageContaining("'target-file-size-bytes' (3) must be > 'min-file-size-bytes' (5)") - .hasMessageContaining("all new files will be smaller than the min threshold"); - - Map invalidTargetMaxSizeOptions = - ImmutableMap.of( - SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "5", - SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, "3"); - assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) - .hasMessageContaining("'target-file-size-bytes' (5) must be < 'max-file-size-bytes' (3)") - .hasMessageContaining("all new files will be larger than the max threshold"); - - Map invalidMinInputFilesOptions = - ImmutableMap.of(SizeBasedFileRewriter.MIN_INPUT_FILES, "0"); - assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) - .hasMessageContaining("'min-input-files' is set to 0 but must be > 0"); - - Map invalidMaxFileGroupSizeOptions = - ImmutableMap.of(SizeBasedFileRewriter.MAX_FILE_GROUP_SIZE_BYTES, "0"); - assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) - .hasMessageContaining("'max-file-group-size-bytes' is set to 0 but must be > 0"); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index f411920a5dcc..24a14bb64d86 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -58,7 +58,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; -import org.apache.iceberg.actions.SizeBasedFileRewriter; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -230,7 +230,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { SparkActions.get(spark) .rewritePositionDeletes(table) - .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") .execute(); table.refresh(); deleteManifestFiles = table.currentSnapshot().deleteManifests(table.io());