Use s3 batch deletes for iceberg expire_snapshots#14434
Use s3 batch deletes for iceberg expire_snapshots#14434findepi merged 3 commits intotrinodb:masterfrom
Conversation
9d3bcd4 to
028042d
Compare
There was a problem hiding this comment.
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
There was a problem hiding this comment.
Thanks. I am aware of this. For now I did this filtering in IcebergMetadata, I will move it here,
75f1d2c to
6fdb097
Compare
|
There are a few permissions edge cases we need to think about. For example if you have two directories:
A user might have permission to delete files in |
Is it possible for table metadata to have files in different locations like this ? |
6fdb097 to
31b85b7
Compare
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Probably it is not. @findepi as this was your idea WDYT ?
There was a problem hiding this comment.
I went through the
hadoop-apachelibrary and it seems that the components of the path which are used in the initialisation of theFileSysteminstance 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?
There was a problem hiding this comment.
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:
There was a problem hiding this comment.
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.
|
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 Grouping the files to remove after scheme could allow us to know which files are on a s3 file system. With The wobbly part in the exposed algorithm is that getting the raw file system may eventually bypass unintentionally some |
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 |
31b85b7 to
a9aa55e
Compare
There was a problem hiding this comment.
I hate the fact that I am doing similar thing twice but I didn't see a way to avoid this repetition..
a9aa55e to
8d9b872
Compare
There was a problem hiding this comment.
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)
There was a problem hiding this comment.
the if (!paths.isEmpty()) condition is redundant.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
I actually thought about this and assumed such a situation is not possible. But sure I will change this
There was a problem hiding this comment.
.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)
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
Because I always struggle to write tests that asserts more than one row :D
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
maybe instead of this -- have the table with 10 snapshots and verify there is only deletion request to MinIO, not 10.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
testExpireSnapshotsBatchDeletes ?
There was a problem hiding this comment.
Should we honor requesterPaysEnabled?
There was a problem hiding this comment.
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
8d9b872 to
bcd3b21
Compare
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 |
a96fad0 to
ccda16b
Compare
There was a problem hiding this comment.
| // deleteFunction should be thread safe if .executeDeleteWith() is used for expireSnapshots | |
| // deleteFunction is not accessed from multiple threads unless .executeDeleteWith() is used |
…o io.trino.filesystem.FileSystemUtils
ccda16b to
6e9bf04
Compare
|
CI: #14686 |
This prevented tests from running, so you need to eg push empty commit to have CI re-run. |
lib/trino-filesystem/src/main/java/io/trino/filesystem/FileSystemUtils.java
Outdated
Show resolved
Hide resolved
lib/trino-hdfs/src/main/java/io/trino/hdfs/TrinoFileSystemCache.java
Outdated
Show resolved
Hide resolved
6e9bf04 to
a35729a
Compare
|
it would be nice to have @electrum 's eyes on filesystem changes, but I know he's busy. |
e375b75 to
89bbb8c
Compare
There was a problem hiding this comment.
Does this work if the file is in the root directory of the file system?
There was a problem hiding this comment.
Then it will be grouped by Path("" ) - is it incorrect ?
There was a problem hiding this comment.
Probably something more like s3://bucket/ in the s3 case at least. I think it's fine
There was a problem hiding this comment.
yes it should be fine imho
89bbb8c to
b729d9b
Compare
|
CI: #14787 |
|
|
||
| public class TrinoS3FileSystem | ||
| extends FileSystem | ||
| implements FileSystemWithBatchDelete |
There was a problem hiding this comment.
Aren't we moving away from use of hadoop's FileSystem towards TrinoFileSystem. Should TrinoFileSystem support batch delete?
There was a problem hiding this comment.
Maybe it should, though currently only HdfsFileSystem implements it and it doesn't support batch delete.
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: