Skip to content

Use s3 batch deletes for iceberg expire_snapshots#14434

Merged
findepi merged 3 commits intotrinodb:masterfrom
homar:homar/use_batch_deletes_for_iceberg_expire_snapshots
Nov 4, 2022
Merged

Use s3 batch deletes for iceberg expire_snapshots#14434
findepi merged 3 commits intotrinodb:masterfrom
homar:homar/use_batch_deletes_for_iceberg_expire_snapshots

Conversation

@homar
Copy link
Copy Markdown
Member

@homar homar commented Oct 3, 2022

Description

Non-technical explanation

Release notes

( ) This is not user-visible or docs only and no release notes are required.
( ) Release notes are required, please propose a release note for me.
( ) Release notes are required, with the following suggested text:

# Section
* Fix some things. ({issue}`issuenumber`)

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

There is a max number of keys allowed per request, if you have more than that you need to split them. There should be a constant defined somewhere but it's 1000 keys https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Thanks. I am aware of this. For now I did this filtering in IcebergMetadata, I will move it here,

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch 5 times, most recently from 75f1d2c to 6fdb097 Compare October 5, 2022 11:18
@alexjo2144
Copy link
Copy Markdown
Member

There are a few permissions edge cases we need to think about. For example if you have two directories:

  • /root/dir-a
  • /root/dir-b

A user might have permission to delete files in dir-a but not dir-b. What should happen if that user tries to batch delete with [/root/dir-a/file-a, /root/dir-b/file-b]

@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 5, 2022

There are a few permissions edge cases we need to think about. For example if you have two directories:

  • /root/dir-a
  • /root/dir-b

A user might have permission to delete files in dir-a but not dir-b. What should happen if that user tries to batch delete with [/root/dir-a/file-a, /root/dir-b/file-b]

Is it possible for table metadata to have files in different locations like this ?
For such a case DeleteObjects would delete the files in dir-a but not in dir-b.
But the previous implementation would have same result. So I think this is ok ?

@homar homar marked this pull request as ready for review October 6, 2022 11:49
@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from 6fdb097 to 31b85b7 Compare October 6, 2022 20:31
Copy link
Copy Markdown
Contributor

@findinpath findinpath Oct 6, 2022

Choose a reason for hiding this comment

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

I went through the hadoop-apache library and it seems that the components of the path which are used in the initialisation of the FileSystem instance are:

  • scheme
  • authority

See org.apache.hadoop.fs.Path#getFileSystem , org.apache.hadoop.fs.FileSystem#get(java.net.URI, org.apache.hadoop.conf.Configuration)

    String scheme = uri.getScheme();
    String authority = uri.getAuthority();

I don't know whether the grouping per directory is actually necessary.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Probably it is not. @findepi as this was your idea WDYT ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I went through the hadoop-apache library and it seems that the components of the path which are used in the initialisation of the FileSystem instance are:

  • scheme
  • authority

Are we talking about contract/interface or implementation?

Also, does https://trino.io/docs/current/connector/hive-s3.html#s3-security-mapping > "prefix" matter?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I missed the fact that the path can be used to add, based on the Path dynamic configuration settings while creating the FileSystem implementation (see DynamicConfigurationProvider) .
So quite likely the whole path is needed.

See s3 predicate matching:

private static Predicate<URI> prefixPredicate(URI prefix)
{
checkArgument("s3".equals(prefix.getScheme()), "prefix URI scheme is not 's3': %s", prefix);
checkArgument(prefix.getQuery() == null, "prefix URI must not contain query: %s", prefix);
checkArgument(prefix.getFragment() == null, "prefix URI must not contain fragment: %s", prefix);
return value -> extractBucketName(prefix).equals(extractBucketName(value)) &&
value.getPath().startsWith(prefix.getPath());
}

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

if we were to fully honor it, we would need to poke into dynamic configuration and see whether all paths in the group would yield the same configs. It's possible (Trino has all the information required), but hard (the information is well interfaced-out and inaccessible). I don't think there is any reasonable real-life use-case to have different configuration for two snapshot files of a table. That's why i deemed grouping by directory should work well in practice.

cc @losipiuk @arhimondr @electrum

@findinpath
Copy link
Copy Markdown
Contributor

I spent some time thinking about this functionality and came up with a few ideas. Hopefully they will be useful for going further with the PR.

Both TrinoFileSystem and HdfsFileSystem do not have at the time of this writing a "batch" delete functionality.
This seems to be a particularity of TrinoS3FileSystem.

Grouping the files to remove after scheme could allow us to know which files are on a s3 file system.

With io.trino.plugin.hive.util.HiveWriteUtils#getRawFileSystem (it needs a path - take the first one from the group) we can know whether we are dealing with a TrinoS3FileSystem. In this specific case pass the files from the group directly to the raw file system to be deleted in batches. Otherwise, use the default TrinoFileSystem#delete method.

The wobbly part in the exposed algorithm is that getting the raw file system may eventually bypass unintentionally some FilterFileSystem which wraps the TrinoS3FileSystem. However, since we are using a method which does not belong to the FileSystem class, I think we are on the safe side.

@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 6, 2022

I spent some time thinking about this functionality and came up with a few ideas. Hopefully they will be useful for going further with the PR.

Both TrinoFileSystem and HdfsFileSystem do not have at the time of this writing a "batch" delete functionality. This seems to be a particularity of TrinoS3FileSystem.

Grouping the files to remove after scheme could allow us to know which files are on a s3 file system.

With io.trino.plugin.hive.util.HiveWriteUtils#getRawFileSystem (it needs a path - take the first one from the group) we can know whether we are dealing with a TrinoS3FileSystem. In this specific case pass the files from the group directly to the raw file system to be deleted in batches. Otherwise, use the default TrinoFileSystem#delete method.

The wobbly part in the exposed algorithm is that getting the raw file system may eventually bypass unintentionally some FilterFileSystem which wraps the TrinoS3FileSystem. However, since we are using a method which does not belong to the FileSystem class, I think we are on the safe side.

Sorry but I am not sure what is the difference between this and the current implementation - apart from obvious thing like I use entire path until last '/' vs use only scheme ?

@homar homar closed this Oct 6, 2022
@homar homar reopened this Oct 6, 2022
@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 6, 2022

I spent some time thinking about this functionality and came up with a few ideas. Hopefully they will be useful for going further with the PR.
Both TrinoFileSystem and HdfsFileSystem do not have at the time of this writing a "batch" delete functionality. This seems to be a particularity of TrinoS3FileSystem.
Grouping the files to remove after scheme could allow us to know which files are on a s3 file system.
With io.trino.plugin.hive.util.HiveWriteUtils#getRawFileSystem (it needs a path - take the first one from the group) we can know whether we are dealing with a TrinoS3FileSystem. In this specific case pass the files from the group directly to the raw file system to be deleted in batches. Otherwise, use the default TrinoFileSystem#delete method.
The wobbly part in the exposed algorithm is that getting the raw file system may eventually bypass unintentionally some FilterFileSystem which wraps the TrinoS3FileSystem. However, since we are using a method which does not belong to the FileSystem class, I think we are on the safe side.

Sorry but I am not sure what is the difference between this and the current implementation - apart from obvious thing like I use entire path until last '/' vs use only scheme ?

Oh wait a second I mixed changes from different PRs, yeap checking down with getRawFile system may be useful here. I will take a look

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from 31b85b7 to a9aa55e Compare October 6, 2022 22:34
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I hate the fact that I am doing similar thing twice but I didn't see a way to avoid this repetition..

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from a9aa55e to 8d9b872 Compare October 7, 2022 06:28
@findepi findepi requested a review from electrum October 7, 2022 07:49
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

call it deleteFiles and put right after deleteFile method.

also, let's have a javadoc explaining whether the operation is supposed to be atomic (it's not)

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

the if (!paths.isEmpty()) condition is redundant.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

what if the path does not contain a slash at all? lastIndexOf will return -1 and substring will fail.
i know in practice it cannot have no slashes, but the intent of this line is not to validate this -- throwing some stupid exception if the caller e.g. erroneously provided a relative path.

you can avoid this with path.replaceFirst("/[^/]*$", ""). I assume regex's overhead is a problem here.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I actually thought about this and assumed such a situation is not possible. But sure I will change this

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

iterate over entry set

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

.stream().findFirst().get() -> .get(0)?

but you're pulling from a wrong collection (i.e. it should be this directory's first path, not the paths first path)

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

remove this since on next line you check it's precisely 1.
instead, assert (earlier) that initialSnapshots.size() is more than 1

btw use assertThat(collection).hasSize, so that in case of failure, the exception message carries the collection contents.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

listagg looks cool, but what is this for here? looks like redundant complexity, especially that we have just 2 rows
let's test with simpler query

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Because I always struggle to write tests that asserts more than one row :D

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

why 2? why size > 0?

document.
also recognizing batch by content-length looks very indirect. is there anything that looks more direct? eg what's the content?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

maybe instead of this -- have the table with 10 snapshots and verify there is only deletion request to MinIO, not 10.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

ehh I thought I explained that during offline discussion. I can create 100 snapshots and there will be 100 events registered by Minio. I didn't manage to see incoming requests I can only see events that are registered and based on that I was not able to find a better solution than this one based on content-length. I was not able to access content itself.

BUT i found a little better way now :D

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

testExpireSnapshotsBatchDeletes ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

move (as in other cases)

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should we honor requesterPaysEnabled?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

yes, definitely, according to aws/aws-sdk-java#1219 that is a problem for single delete it should work for batch delete so I will simply set it

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from 8d9b872 to bcd3b21 Compare October 7, 2022 19:26
@findepi findepi requested review from electrum and removed request for electrum October 18, 2022 07:56
@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 18, 2022

in a threaded context many things go wrong

ArrayList can get corrupted, we may loose files
pathsToDelete.clear(); can be invoked with some paths never sent to deletion
deleteFiles can be invoked twice on same/similar sets of paths and may result in "file missing so cannot delete" exception

As I explained above, currently I assume this is run with a single thread. I am aware of all these problems if it would be called by many threads

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from a96fad0 to ccda16b Compare October 18, 2022 11:31
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Suggested change
// deleteFunction should be thread safe if .executeDeleteWith() is used for expireSnapshots
// deleteFunction is not accessed from multiple threads unless .executeDeleteWith() is used

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from ccda16b to 6e9bf04 Compare October 19, 2022 12:33
@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 19, 2022

CI: #14686

@homar homar requested a review from findepi October 19, 2022 14:34
@findepi
Copy link
Copy Markdown
Member

findepi commented Oct 25, 2022

CI: #14686

This prevented tests from running, so you need to eg push empty commit to have CI re-run.

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from 6e9bf04 to a35729a Compare October 25, 2022 11:38
@findepi
Copy link
Copy Markdown
Member

findepi commented Oct 25, 2022

it would be nice to have @electrum 's eyes on filesystem changes, but I know he's busy.
Will merge sometime later this week.

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch 2 times, most recently from e375b75 to 89bbb8c Compare October 25, 2022 19:01
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Does this work if the file is in the root directory of the file system?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Then it will be grouped by Path("" ) - is it incorrect ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Probably something more like s3://bucket/ in the s3 case at least. I think it's fine

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

yes it should be fine imho

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

ICEBERG_FILESYSTEM_ERROR ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

ICEBERG_FILESYSTEM_ERROR ?

@homar homar force-pushed the homar/use_batch_deletes_for_iceberg_expire_snapshots branch from 89bbb8c to b729d9b Compare October 26, 2022 19:55
@homar
Copy link
Copy Markdown
Member Author

homar commented Oct 27, 2022

CI: #14787


public class TrinoS3FileSystem
extends FileSystem
implements FileSystemWithBatchDelete
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Aren't we moving away from use of hadoop's FileSystem towards TrinoFileSystem. Should TrinoFileSystem support batch delete?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Maybe it should, though currently only HdfsFileSystem implements it and it doesn't support batch delete.

@findepi findepi merged commit ea02559 into trinodb:master Nov 4, 2022
@findepi findepi mentioned this pull request Nov 4, 2022
@github-actions github-actions bot added this to the 403 milestone Nov 4, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Development

Successfully merging this pull request may close these issues.

6 participants