-
Notifications
You must be signed in to change notification settings - Fork 2.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Spark 3.4: Implement rewrite position deletes #7389
Changes from 1 commit
372aba4
d19fa6a
9092266
4bc6b21
f3d4cb2
9b3709e
1c27cb2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -40,9 +40,9 @@ | |
import org.apache.iceberg.Table; | ||
import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles; | ||
import org.apache.iceberg.actions.RewritePositionDeleteFiles; | ||
import org.apache.iceberg.actions.RewritePositionDeleteGroup; | ||
import org.apache.iceberg.actions.RewritePositionDeletesCommitManager; | ||
import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService; | ||
import org.apache.iceberg.actions.RewritePositionDeletesGroup; | ||
import org.apache.iceberg.exceptions.CommitFailedException; | ||
import org.apache.iceberg.exceptions.ValidationException; | ||
import org.apache.iceberg.expressions.Expression; | ||
|
@@ -68,19 +68,19 @@ | |
import org.slf4j.LoggerFactory; | ||
|
||
/** Spark implementation of {@link RewritePositionDeleteFiles}. */ | ||
public class RewritePositionDeletesSparkAction | ||
extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction> | ||
public class RewritePositionDeleteSparkAction | ||
extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction> | ||
implements RewritePositionDeleteFiles { | ||
|
||
private static final Logger LOG = | ||
LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class); | ||
private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class); | ||
private static final Set<String> VALID_OPTIONS = | ||
ImmutableSet.of( | ||
MAX_CONCURRENT_FILE_GROUP_REWRITES, | ||
PARTIAL_PROGRESS_ENABLED, | ||
PARTIAL_PROGRESS_MAX_COMMITS, | ||
REWRITE_JOB_ORDER); | ||
private static final Result EMPTY = ImmutableRewritePositionDeleteFiles.Result.builder().build(); | ||
private static final Result EMPTY_RESULT = | ||
ImmutableRewritePositionDeleteFiles.Result.builder().build(); | ||
|
||
private final Table table; | ||
private final SparkBinPackPositionDeletesRewriter rewriter; | ||
|
@@ -90,27 +90,27 @@ public class RewritePositionDeletesSparkAction | |
private boolean partialProgressEnabled; | ||
private RewriteJobOrder rewriteJobOrder; | ||
|
||
RewritePositionDeletesSparkAction(SparkSession spark, Table table) { | ||
RewritePositionDeleteSparkAction(SparkSession spark, Table table) { | ||
super(spark); | ||
this.table = table; | ||
this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table); | ||
} | ||
|
||
@Override | ||
protected RewritePositionDeletesSparkAction self() { | ||
protected RewritePositionDeleteSparkAction self() { | ||
return this; | ||
} | ||
|
||
@Override | ||
public RewritePositionDeletesSparkAction filter(Expression expression) { | ||
public RewritePositionDeleteSparkAction filter(Expression expression) { | ||
throw new UnsupportedOperationException("Regular filters not supported yet."); | ||
} | ||
|
||
@Override | ||
public RewritePositionDeleteFiles.Result execute() { | ||
if (table.currentSnapshot() == null) { | ||
LOG.info("Nothing found to rewrite in empty table {}", table.name()); | ||
return EMPTY; | ||
return EMPTY_RESULT; | ||
} | ||
|
||
validateAndInitOptions(); | ||
|
@@ -120,10 +120,10 @@ public RewritePositionDeleteFiles.Result execute() { | |
|
||
if (ctx.totalGroupCount() == 0) { | ||
LOG.info("Nothing found to rewrite in {}", table.name()); | ||
return EMPTY; | ||
return EMPTY_RESULT; | ||
} | ||
|
||
Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition); | ||
Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition); | ||
|
||
RewritePositionDeletesCommitManager commitManager = commitManager(); | ||
if (partialProgressEnabled) { | ||
|
@@ -147,9 +147,7 @@ private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() { | |
StructLikeMap.create(partitionType); | ||
|
||
for (PositionDeletesScanTask task : scanTasks) { | ||
StructLike taskPartition = task.file().partition(); | ||
StructLike coerced = | ||
PartitionUtil.coercePartition(partitionType, task.spec(), taskPartition); | ||
StructLike coerced = coercePartition(task, partitionType); | ||
|
||
List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced); | ||
if (partitionTasks == null) { | ||
|
@@ -168,7 +166,6 @@ private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() { | |
rewriter.planFileGroups(partitionTasks); | ||
List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups); | ||
if (groups.size() > 0) { | ||
// use coerced partition for map key uniqueness, but return original partition | ||
fileGroupsByPartition.put(partition, groups); | ||
} | ||
}); | ||
|
@@ -184,8 +181,8 @@ private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() { | |
} | ||
|
||
@VisibleForTesting | ||
RewritePositionDeleteGroup rewriteDeleteFiles( | ||
RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) { | ||
RewritePositionDeletesGroup rewriteDeleteFiles( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just making sure that this is indeed used for testing. |
||
RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) { | ||
String desc = jobDesc(fileGroup, ctx); | ||
Set<DeleteFile> addedFiles = | ||
withJobGroupInfo( | ||
|
@@ -214,14 +211,14 @@ RewritePositionDeletesCommitManager commitManager() { | |
|
||
private Result doExecute( | ||
RewriteExecutionContext ctx, | ||
Stream<RewritePositionDeleteGroup> groupStream, | ||
Stream<RewritePositionDeletesGroup> groupStream, | ||
RewritePositionDeletesCommitManager commitManager) { | ||
ExecutorService rewriteService = rewriteService(); | ||
|
||
ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups = | ||
ConcurrentLinkedQueue<RewritePositionDeletesGroup> rewrittenGroups = | ||
Queues.newConcurrentLinkedQueue(); | ||
|
||
Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder = | ||
Tasks.Builder<RewritePositionDeletesGroup> rewriteTaskBuilder = | ||
Tasks.foreach(groupStream) | ||
.executeWith(rewriteService) | ||
.stopOnFailure() | ||
|
@@ -248,9 +245,7 @@ private Result doExecute( | |
rewrittenGroups.size(), | ||
e); | ||
|
||
Tasks.foreach(rewrittenGroups) | ||
.suppressFailureWhenFinished() | ||
.run(commitManager::abortFileGroup); | ||
Tasks.foreach(rewrittenGroups).suppressFailureWhenFinished().run(commitManager::abort); | ||
throw e; | ||
} finally { | ||
rewriteService.shutdown(); | ||
|
@@ -273,7 +268,7 @@ private Result doExecute( | |
|
||
List<FileGroupRewriteResult> rewriteResults = | ||
rewrittenGroups.stream() | ||
.map(RewritePositionDeleteGroup::asResult) | ||
.map(RewritePositionDeletesGroup::asResult) | ||
.collect(Collectors.toList()); | ||
|
||
return ImmutableRewritePositionDeleteFiles.Result.builder() | ||
|
@@ -283,7 +278,7 @@ private Result doExecute( | |
|
||
private Result doExecuteWithPartialProgress( | ||
RewriteExecutionContext ctx, | ||
Stream<RewritePositionDeleteGroup> groupStream, | ||
Stream<RewritePositionDeletesGroup> groupStream, | ||
RewritePositionDeletesCommitManager commitManager) { | ||
ExecutorService rewriteService = rewriteService(); | ||
|
||
|
@@ -305,7 +300,7 @@ private Result doExecuteWithPartialProgress( | |
|
||
// Stop Commit service | ||
commitService.close(); | ||
List<RewritePositionDeleteGroup> commitResults = commitService.results(); | ||
List<RewritePositionDeletesGroup> commitResults = commitService.results(); | ||
if (commitResults.size() == 0) { | ||
LOG.error( | ||
"{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " | ||
|
@@ -317,17 +312,17 @@ private Result doExecuteWithPartialProgress( | |
|
||
List<FileGroupRewriteResult> rewriteResults = | ||
commitResults.stream() | ||
.map(RewritePositionDeleteGroup::asResult) | ||
.map(RewritePositionDeletesGroup::asResult) | ||
.collect(Collectors.toList()); | ||
return ImmutableRewritePositionDeleteFiles.Result.builder() | ||
.rewriteResults(rewriteResults) | ||
.build(); | ||
} | ||
|
||
Stream<RewritePositionDeleteGroup> toGroupStream( | ||
Stream<RewritePositionDeletesGroup> toGroupStream( | ||
RewriteExecutionContext ctx, | ||
Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) { | ||
Stream<RewritePositionDeleteGroup> rewriteFileGroupStream = | ||
Stream<RewritePositionDeletesGroup> rewriteFileGroupStream = | ||
groupsByPartition.entrySet().stream() | ||
.flatMap( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. for future: Can we try refactoring this using some helper methods cause Spotless formats this in a weird way. Not in this PR. |
||
e -> { | ||
|
@@ -338,18 +333,17 @@ Stream<RewritePositionDeleteGroup> toGroupStream( | |
tasks -> { | ||
int globalIndex = ctx.currentGlobalIndex(); | ||
int partitionIndex = ctx.currentPartitionIndex(partition); | ||
// as coerced partition used for map, return original partition | ||
FileGroupInfo info = | ||
ImmutableRewritePositionDeleteFiles.FileGroupInfo.builder() | ||
.globalIndex(globalIndex) | ||
.partitionIndex(partitionIndex) | ||
.partition(partition) | ||
.build(); | ||
return new RewritePositionDeleteGroup(info, tasks); | ||
return new RewritePositionDeletesGroup(info, tasks); | ||
}); | ||
}); | ||
|
||
return rewriteFileGroupStream.sorted(RewritePositionDeleteGroup.comparator(rewriteJobOrder)); | ||
return rewriteFileGroupStream.sorted(RewritePositionDeletesGroup.comparator(rewriteJobOrder)); | ||
} | ||
|
||
private void validateAndInitOptions() { | ||
|
@@ -399,7 +393,7 @@ private void validateAndInitOptions() { | |
PARTIAL_PROGRESS_ENABLED); | ||
} | ||
|
||
private String jobDesc(RewritePositionDeleteGroup group, RewriteExecutionContext ctx) { | ||
private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContext ctx) { | ||
StructLike partition = group.info().partition(); | ||
if (partition.size() > 0) { | ||
return String.format( | ||
|
@@ -455,4 +449,8 @@ public int totalGroupCount() { | |
return totalGroupCount; | ||
} | ||
} | ||
|
||
private StructLike coercePartition(PositionDeletesScanTask task, StructType partitionType) { | ||
return PartitionUtil.coercePartition(partitionType, task.spec(), task.partition()); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't this be called
RewritePositionDeleteFilesSparkAction
? This is public facing and we usually name it as the interface name +SparkAction
.