Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions core/src/main/java/org/apache/iceberg/util/StructLikeMap.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
Expand Down Expand Up @@ -171,4 +172,10 @@ public R setValue(R value) {
throw new UnsupportedOperationException("Does not support setValue.");
}
}

public <U> StructLikeMap<U> transformValues(Function<T, U> func) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Your idea of making this a static method also makes sense to me. It would be more in line with Guava and other similar libs. If you decide to make it static, keep in mind the ordering of methods. Static methods should be right after the constructor.

public static <T, U> StructLikeMap<U> transformValues(StructLikeMap<T> map, Function<T, U> func) {
  ...
}

StructLikeMap<U> result = create(type);
wrapperMap.forEach((key, value) -> result.put(key.get(), func.apply(value)));
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
Expand All @@ -68,11 +67,12 @@
import org.slf4j.LoggerFactory;

/** Spark implementation of {@link RewritePositionDeleteFiles}. */
public class RewritePositionDeleteSparkAction
extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
public class RewritePositionDeleteFilesSparkAction
extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteFilesSparkAction>
implements RewritePositionDeleteFiles {

private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
private static final Logger LOG =
LoggerFactory.getLogger(RewritePositionDeleteFilesSparkAction.class);
private static final Set<String> VALID_OPTIONS =
ImmutableSet.of(
MAX_CONCURRENT_FILE_GROUP_REWRITES,
Expand All @@ -90,19 +90,19 @@ public class RewritePositionDeleteSparkAction
private boolean partialProgressEnabled;
private RewriteJobOrder rewriteJobOrder;

RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) {
super(spark);
this.table = table;
this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
}

@Override
protected RewritePositionDeleteSparkAction self() {
protected RewritePositionDeleteFilesSparkAction self() {
return this;
}

@Override
public RewritePositionDeleteSparkAction filter(Expression expression) {
public RewritePositionDeleteFilesSparkAction filter(Expression expression) {
throw new UnsupportedOperationException("Regular filters not supported yet.");
}

Expand All @@ -115,7 +115,7 @@ public RewritePositionDeleteFiles.Result execute() {

validateAndInitOptions();

Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);

if (ctx.totalGroupCount() == 0) {
Expand All @@ -125,63 +125,68 @@ public RewritePositionDeleteFiles.Result execute() {

Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);

RewritePositionDeletesCommitManager commitManager = commitManager();
if (partialProgressEnabled) {
return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
return doExecuteWithPartialProgress(ctx, groupStream, commitManager());
} else {
return doExecute(ctx, groupStream, commitManager);
return doExecute(ctx, groupStream, commitManager());
}
}

private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Optional: A note to the above snippet. What about getting rid of commitManager var and just calling the method directly?

if (partialProgressEnabled) {
  return doExecuteWithPartialProgress(ctx, groupStream, commitManager());
} else {
  return doExecute(ctx, groupStream, commitManager());
}

Table deletesTable =
MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
CloseableIterable<PositionDeletesScanTask> scanTasks =
CloseableIterable.transform(
deletesTable.newBatchScan().ignoreResiduals().planFiles(),
t -> (PositionDeletesScanTask) t);
private StructLikeMap<List<List<PositionDeletesScanTask>>> planFileGroups() {
CloseableIterable<PositionDeletesScanTask> fileTasks = planFiles();

try {
StructType partitionType = Partitioning.partitionType(table);
StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
StructLikeMap.create(partitionType);

for (PositionDeletesScanTask task : scanTasks) {
StructLike coerced = coercePartition(task, partitionType);

List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
if (partitionTasks == null) {
partitionTasks = Lists.newArrayList();
}
partitionTasks.add(task);
filesByPartition.put(coerced, partitionTasks);
}

StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
StructLikeMap.create(partitionType);

filesByPartition.forEach(
(partition, partitionTasks) -> {
Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
rewriter.planFileGroups(partitionTasks);
List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
if (groups.size() > 0) {
fileGroupsByPartition.put(partition, groups);
}
});

return fileGroupsByPartition;
StructLikeMap<List<PositionDeletesScanTask>> fileTasksByPartition =
groupByPartition(partitionType, fileTasks);
return fileGroupsByPartition(fileTasksByPartition);
} finally {
try {
scanTasks.close();
fileTasks.close();
} catch (IOException io) {
LOG.error("Cannot properly close file iterable while planning for rewrite", io);
}
}
}

@VisibleForTesting
RewritePositionDeletesGroup rewriteDeleteFiles(
private CloseableIterable<PositionDeletesScanTask> planFiles() {
Table deletesTable =
MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);

return CloseableIterable.transform(
deletesTable.newBatchScan().ignoreResiduals().planFiles(),
task -> (PositionDeletesScanTask) task);
}

private StructLikeMap<List<PositionDeletesScanTask>> groupByPartition(
StructType partitionType, Iterable<PositionDeletesScanTask> tasks) {
StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
StructLikeMap.create(partitionType);

for (PositionDeletesScanTask task : tasks) {
StructLike coerced = coercePartition(task, partitionType);

List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
if (partitionTasks == null) {
partitionTasks = Lists.newArrayList();
}
partitionTasks.add(task);
filesByPartition.put(coerced, partitionTasks);
}

return filesByPartition;
}

private StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition(
StructLikeMap<List<PositionDeletesScanTask>> filesByPartition) {
return filesByPartition.transformValues(this::planFileGroups);
}

private List<List<PositionDeletesScanTask>> planFileGroups(List<PositionDeletesScanTask> tasks) {
return ImmutableList.copyOf(rewriter.planFileGroups(tasks));
}

private RewritePositionDeletesGroup rewriteDeleteFiles(
RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) {
String desc = jobDesc(fileGroup, ctx);
Set<DeleteFile> addedFiles =
Expand All @@ -204,8 +209,7 @@ private ExecutorService rewriteService() {
.build()));
}

@VisibleForTesting
RewritePositionDeletesCommitManager commitManager() {
private RewritePositionDeletesCommitManager commitManager() {
return new RewritePositionDeletesCommitManager(table);
}

Expand Down Expand Up @@ -282,12 +286,12 @@ private Result doExecuteWithPartialProgress(
RewritePositionDeletesCommitManager commitManager) {
ExecutorService rewriteService = rewriteService();

// Start Commit Service
// start commit service
int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
CommitService commitService = commitManager.service(groupsPerCommit);
commitService.start();

// Start rewrite tasks
// start rewrite tasks
Tasks.foreach(groupStream)
.suppressFailureWhenFinished()
.executeWith(rewriteService)
Expand All @@ -298,7 +302,7 @@ private Result doExecuteWithPartialProgress(
.run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
rewriteService.shutdown();

// Stop Commit service
// stop commit service
commitService.close();
List<RewritePositionDeletesGroup> commitResults = commitService.results();
if (commitResults.size() == 0) {
Expand All @@ -319,31 +323,31 @@ private Result doExecuteWithPartialProgress(
.build();
}

Stream<RewritePositionDeletesGroup> toGroupStream(
private Stream<RewritePositionDeletesGroup> toGroupStream(
RewriteExecutionContext ctx,
Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
Stream<RewritePositionDeletesGroup> rewriteFileGroupStream =
groupsByPartition.entrySet().stream()
.flatMap(
e -> {
StructLike partition = e.getKey();
List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
return scanGroups.stream()
.map(
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);
});
});

return rewriteFileGroupStream.sorted(RewritePositionDeletesGroup.comparator(rewriteJobOrder));
return groupsByPartition.entrySet().stream()
.filter(e -> e.getValue().size() != 0)
.flatMap(
e -> {
StructLike partition = e.getKey();
List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
return scanGroups.stream().map(tasks -> newRewriteGroup(ctx, partition, tasks));
})
.sorted(RewritePositionDeletesGroup.comparator(rewriteJobOrder));
}

private RewritePositionDeletesGroup newRewriteGroup(
RewriteExecutionContext ctx, StructLike partition, List<PositionDeletesScanTask> 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() {
Expand Down Expand Up @@ -418,16 +422,14 @@ private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContex
}

static class RewriteExecutionContext {
private final Map<StructLike, Integer> numGroupsByPartition;
private final StructLikeMap<Integer> numGroupsByPartition;
private final int totalGroupCount;
private final Map<StructLike, Integer> partitionIndexMap;
private final AtomicInteger groupIndex;

RewriteExecutionContext(
Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
this.numGroupsByPartition =
groupsByPartition.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size()));
StructLikeMap<List<List<PositionDeletesScanTask>>> 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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocat
}

@Override
public RewritePositionDeleteSparkAction rewritePositionDeletes(Table table) {
return new RewritePositionDeleteSparkAction(spark, table);
public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) {
return new RewritePositionDeleteFilesSparkAction(spark, table);
}
}