Skip to content

Conversation

@szehon-ho
Copy link
Member

@szehon-ho szehon-ho commented Jan 13, 2023

This adds an action to cleanup dangling (invalid) DeleteFiles that may otherwise keep getting carried over with the table's current snapshot and which may negatively impact read performance.

The problem and design doc is here: https://docs.google.com/document/d/11d-cIUR_89kRsMmWnEoxXGZCvp7L4TUmPJqUC60zB5M/edit#

In a nutshell, the current table-wide mechanism is crude and may miss many instances of aging off DeleteFiles, even after they become invalid after compaction. This implements a spark action to perform a partition-by-partition removal using the same rules.

@szehon-ho szehon-ho force-pushed the remove_dangling_delete branch from 8f59b01 to 88a05db Compare January 13, 2023 21:03
.toDF("partition", "spec_id", "min_data_sequence_number");

// Dangling position delete files
Column joinCond =
Copy link
Member Author

Choose a reason for hiding this comment

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

I can't reproduce it on my own machine, but on build machine I get the error if I try the cleaner: Dataset.join(ds, Seq columns)

error: no suitable method found for join(Dataset<Row>,Buffer<String>) see: https://github.com/apache/iceberg/actions/runs/3913210764/jobs/6688831726

So I make an explicit join condition, like RewriteManifestFileSparkAction

import org.apache.iceberg.DeleteFile;

/**
* An action that removes dangling delete files from the current snapshot. A delete file is dangling
Copy link
Member

@RussellSpitzer RussellSpitzer Jan 17, 2023

Choose a reason for hiding this comment

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

I think we need a note that this removes delete files only if they don't apply to any non-expired datafile. Just to make it clear that this isn't just about "live" delete files

* <p>The following dangling delete files are removed:
*
* <ul>
* <li>Position delete files with a sequence number less than that of any data file in the same
Copy link
Member

Choose a reason for hiding this comment

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

This is more of a technical detail right? Not sure we need it in the java doc


public class RemoveDanglingDeleteFilesActionResult implements RemoveDanglingDeleteFiles.Result {

private static final RemoveDanglingDeleteFilesActionResult EMPTY =
Copy link
Member

Choose a reason for hiding this comment

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

Do we really need this? Seems like in the code we can just add

new RemoveDanglingDeleteFileActionResult(Collections.emptyList()) 

at
https://github.com/apache/iceberg/pull/6581/files#diff-afa01360c6badf264da62497ccb1186cdd841cae290ab15b0a2b086cc100b9caR79

Doesn't seem like we really are making that many of these objects

if (useCaching) {
reusableDS = ds.cache();
} else {
int parallelism = SQLConf.get().numShufflePartitions();
Copy link
Member

Choose a reason for hiding this comment

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

This is a bit internal to spark, probably fine but i'm not a fan of touching this over spark.conf()

} else {
int parallelism = SQLConf.get().numShufflePartitions();
reusableDS =
ds.repartition(parallelism).map((MapFunction<T, T>) value -> value, ds.exprEnc());
Copy link
Member

@RussellSpitzer RussellSpitzer Jan 17, 2023

Choose a reason for hiding this comment

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

why do we repartition here? (and encode)

@szehon-ho
Copy link
Member Author

szehon-ho commented Jan 18, 2023

Chatting with @aokolnychyi , @RussellSpitzer , to do an analysis on when this can be useful.

There will be two types of operations that can remove delete files:

Operation Cost File Type Description
RemoveDanglingDeletes (this one) Metadata-Only, cost will be like querying files/partition table Both Removes position deletes with sequence number less than that of the min sequence number of all data files in each partition
RewritePositionDeletes (to be developed) Data-operation, need to read/write all concerned delete files Position only (Equality Deletes will need to be converted to PositionDeletes) Read all position delete files satisfying given filter, write them back out , filtering out position delete entries that refer to data files that no longer exist

Analysis: RemoveDanglingDeleteFiles is cheaper and simpler, and can work across both types of files out of the box. However, to get it to exactly work, we need the following conditions: RewriteDataFiles being run with:

  • Filter that includes entire partition(s)
  • All data files in the partition with delete files gets rewritten, ie any of these:
    • rewrite-all=true
    • delete-file-threshold=1
    • All data files happen to meet the criteria of rewrite without these flags.
  • 'use-starting-sequence-number' needs to be false. This is to properly identify old delete files as invalid using sequence number rule. This is only needed for position-deletes, as equality-deletes are not applied to equivalent sequence number.

Note RemoveDanglingDeleteFiles can still remove some delete files if these conditions are not met, but just it may not do so for all dangling delete files, because an old data file (one with a low sequence number) not rewritten in a partition will prevent delete files from getting removed.

So Im open to whether there is a good use-case of this. One idea is to bundle this with RewriteDataFiles, and if trigger optimistically if these conditions are met, or trigger in any case in hopes it will remove delete files as, as its relatively cheap.

Otherwise, the complete solution (all to be developed) would be:
For position deletes, run RewritePositionDeletes across all partitions
For equality deletes, run ConvertToPosDeletes, then RewritePositionDeletes across all partitions.

"data_file.spec_id as spec_id",
"data_file.file_path as file_path",
"data_file.content as content",
"data_file.file_size_in_bytes as file_size_in_bytes",
Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry if it's a naive question, do we need to project file_size_in_bytes for this action?

Copy link
Contributor

Choose a reason for hiding this comment

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

Nvm, we need to serialize the result to a delete file so it's needed.


/** The action result that contains a summary of the execution. */
interface Result {
/** Removes representation of removed delete files. */
Copy link
Contributor

Choose a reason for hiding this comment

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

Should this comment just be "List of removed delete files"?

Comment on lines +204 to +206
for (int i = 0; i < partitionRow.length(); i++) {
partition.set(i, partitionRow.get(i));
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Style nit: newline after the loop

@amogh-jahagirdar
Copy link
Contributor

amogh-jahagirdar commented Jan 22, 2023

@szehon-ho Thanks for the detailed analysis. On the surface it does make sense to combine with existing compaction mechanisms like RewriteDataFiles if the metadata only RemoveDanglingDeleteFiles is cheap and especially if many users are already running RewriteDataFiles periodically anyways. Is there a case we're missing where users would just want to run remove dangling delete files separately? I can't really think of a case.

}

String desc = String.format("Remove dangling delete files for %s", table.name());
JobGroupInfo info = newJobGroupInfo("REWRITE-MANIFESTS", desc);
Copy link
Contributor

Choose a reason for hiding this comment

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

[minor] should we also mention it's rewriting manifest post removing danling delete files in the name ?

Copy link
Member Author

Choose a reason for hiding this comment

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

Good point.

Comment on lines +127 to +132
Dataset<Row> minDataSeqNumberPerPartition =
entries
.filter("content == 0") // data files
.groupBy("partition", "spec_id")
.agg(min("sequence_number"))
.toDF("partition", "spec_id", "min_data_sequence_number");
Copy link
Contributor

Choose a reason for hiding this comment

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

can we cache this df this as well, since we need this in two joins below ?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yea I am not so sure, it seems cache is a bit controversial. I think the general approach in Iceberg spark action is not to use cache?

@manuzhang
Copy link
Member

manuzhang commented Feb 1, 2023

One idea is to bundle this with RewriteDataFiles, and if trigger optimistically if these conditions are met, or trigger in any case in hopes it will remove delete files as, as its relatively cheap.

@szehon-ho Have you considered partial-progress.enabled=true? Currently, delete files can't be removed when some file groups (partitions) are compacted and committed.

@szehon-ho
Copy link
Member Author

Thanks all for reviews. After thinking, this will be good to have, but as the complete 100% position delete removal will be done via minor compaction of delete files (first part of the effort is here: #6365, more to come). Will work on that first.

@amogh-jahagirdar yea I initially thought it would be useful as a standalone action, but some chats with @aokolnychyi and maybe its a bit too tricky for user to know when to run it. Still open though.

@manuzhang do you mean integrating this commit with existing partial commits? Initially envisoning a separate commit altogether at the end of RewriteDataFiles.

@manuzhang
Copy link
Member

manuzhang commented Feb 2, 2023

@szehon-ho yes, integrating removing delete files per partition with partial commits. When new position delete files are generated during RewriteDataFiles with partial commits, all following commits will fail and no delete files will be removed at the end.

@eric666666
Copy link

Is this action currently available? Is there any usage documentation.

@eric666666
Copy link

Is this action currently available? Is there any usage documentation.

We have already applied the v2 table to production. It is too uncomfortable to delete the delete file when the snapshot expires. When will this Pr be merged into the master.

@zinking
Copy link
Contributor

zinking commented May 15, 2023

I am +1 for integrating these into existing Rewrite Action.

@szehon-ho
Copy link
Member Author

szehon-ho commented May 15, 2023

Hi, can you guys please check #7389 , it is already merged. It is a way to do it manually for now (and also optimize position delete), we can come back later to integrate this action into rewriteDataFiles automatically if we need. There's also some ongoing post-pr improvements thats also in review like #7582 and #7572


return builder
.withPath(path)
.withPartition(partition)
Copy link
Contributor

Choose a reason for hiding this comment

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

the partition data here doesn't comply with the builder spec, so it will build out incorrect partition data.

is it a bit of overwork to rebuild the deleteFile, cant we just delete using paths? a lot of overhead could be avoided.

Copy link
Contributor

Choose a reason for hiding this comment

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

usually the copy needs to assert the specs are the same

    public Builder copy(DeleteFile toCopy) {
      if (isPartitioned) {
        Preconditions.checkState(
            specId == toCopy.specId(), "Cannot copy a DeleteFile with a different spec");
        this.partitionData = DataFiles.copyPartitionData(spec, toCopy.partition(), partitionData);
      }

Copy link
Contributor

Choose a reason for hiding this comment

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

it would be great if we can delete a deleteFile by filePaths only, but from mergingSnapshotProducer, looks like we never make it to work for path based deletion

protected void delete(DeleteFile file) {
deleteFilterManager.delete(file);
}
/** Add a specific data path to be deleted in the new snapshot. */
protected void delete(CharSequence path) {
// this is an old call that never worked for delete files and can only be used to remove data
// files.
filterManager.delete(path);
}

@github-actions
Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Aug 24, 2024
@github-actions
Copy link

This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

@github-actions github-actions bot closed this Sep 12, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants