-
Notifications
You must be signed in to change notification settings - Fork 3k
Core: Move deleted files to Hadoop trash if configured #14501
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
Conversation
b0ba8b9 to
8d07d49
Compare
8d07d49 to
5cb16cf
Compare
anuragmantri
left a comment
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.
Thanks for the PR @jordepic. This is very useful.
However, it seems like a behavior change (even if trash was enabled previously, Iceberg was not honoring it). IMO, we should make this configurable using a property to avoid surprises (unexpected storage consumption).
| } | ||
|
|
||
| private void deletePath(FileSystem fs, Path toDelete, boolean recursive) throws IOException { | ||
| Trash trash = new Trash(fs, getConf()); |
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.
I'm concerned about number of Trash objects we create. Does Hadoop API ensure we can reuse the trash object for a given (fs, conf)?
I couldn't tell from https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/fs/Trash.html#Trash-org.apache.hadoop.fs.FileSystem-org.apache.hadoop.conf.Configuration-
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.
It's a good call. I've added a new toggle that can be put in the hadoop configuration to determine if we want to use the trash for iceberg, following Russel Spitzer's example in other HadoopFileIO changes.
I've taken a look regarding object reuse. The trash can change due to lots of changes in configuration (meaning I'd have to create a cache based on 5+ configuration values which are susceptible to change in the future), unlike the file system (Key doesn't actually rely on conf, just relies on the URI and user group information). With that being said, the change that I made to check for hadoop configuration first makes it so that we don't create the Trash object unless specifically opted into. I hope that this is good enough for now - an iceberg user will now have to opt into this change to experience any possible object churn.
5cb16cf to
efc6a8f
Compare
| return; | ||
| } | ||
| Trash trash = new Trash(fs, getConf()); | ||
| if (!trash.isEnabled()) { |
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.
If you can configure the trash at the Hadoop Conf level, why are we adding a separate configuration? Shouldn't we just do this when the HadoopFileIO is initialized and rely on the Hadoop trash conf? It feels like we're adding two separate configs to enable trash.
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.
Fair enough. A previous commenter here wanted to do this in order to make the trash "opt-in", not "opt-out", for those who had already configured it.
I agree though, if trash is configured the normal way, we should use it.
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.
Sorry, my understanding was that the current java API for delete always "ignores" the trash config even if it is set server side. In which case, not ignoring it anymore could be potentially unexpected. But I guess that is the right thing to do (honor the trash config if it is set via Hadoop conf). I'm ok with just using the Hadoop conf. Thanks!
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.
I'm not particularly familiar with the way the Hadoop Trash works, but it's quite odd to me. The Trash isn't natively part of the FileSystem (like versioned objects are native to cloud storage). This leaves an awkward situation where, if we respect the core-site.xml config, then we turn it on everywhere like you said @anuragmantri (assuming the deleter has access to the trash?).
I could see this going both ways in that if you configure the core-site.xml and didn't realize there was a second property to set, you would lose data. You also potentially have the issue where deleted data goes to the trash configured for whoever deleted it (so two separate deletes could end up in completely different locations).
I'm not sure what's right here and open to suggestions.
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.
I could see this going both ways in that if you configure the core-site.xml and didn't realize there was a second property to set, you would lose data.
To be fair, this is the status quo
You also potentially have the issue where deleted data goes to the trash configured for whoever deleted it (so two separate deletes could end up in completely different locations).
I suppose that would be the case for HDFS trash in general - even if writing parquet files without iceberg, each writer might potentially use a different trash based on configuration.
I think these are just limitations of the existing system, but let me know if you disagree
|
@jordepic I'm a little concerned about the utility of this. If we're just relocating arbitrary files into the trash location, how do you know which table it was associated with? In isolation it seems like it would make sense, but across a warehouse, this feels like it would be really difficult to reconstruct anything. |
efc6a8f to
ba28083
Compare
@danielcweeks a file at path /iceberg/tablename/data/.... is relocated to /.Trash/current/iceberg/tablename/data/... It doesn't go to a completely arbitrary path! |
ba28083 to
d0d62e8
Compare
I agree that this may not very useful in isolation, but can we still let the client use the trash in HadoopIO (if configured) and have the ability for users to restore the table state. We have had examples where users are able to restore accidentally deleted files via cloud providers' object lifecycle policies but could not do so in Hadoop environments because the client was not using the trash. |
|
it would be nice to provide a table level parameter to control this behavior |
@ludlows That was basically the first iteration of my change. I think that @danielcweeks felt this level of control was unnecessary, and that those that configure their hadoop to use the trash should use it. Open to more discussion here. |
|
@jordepic and @ludlows After looking a little more into the way trash works, I don't think this is something we want to turn on at a table level (especially considering how this implementation works). The Trash feature in Hadoop/HDFS is quite strange as it's a client, config, and cluster level feature that all depend upon each other. For example, the client has to respect the config and initialize the Trash and perform a move operation otherwise it's ignored. The config has to be set and configured properly to a location the user has access to. Finally, if you don't apply the configuration to both the client and the NameNode, then cleanup won't be performed properly. Given all of that, this feels very much like a administrator-level feature that needs to be configured (this appears to be the case for Cloudera already, though I don't know if engines like Hive/Impala respect the trash settings). It could be potentially dangerous to allow users to configure this on a per-table basis because cleanup may not be configured, which may result in data that should be deleted, persisting in the file system. There's also nothing that appears to prevent the configuration from being applied to other file-system implementations (like S3A), which would be bad (data copy, no cleanup), but I feel like we should discourage that. @jordepic Is there anything we can do to prevent this? I'm not a huge fan of this approach, but it seems like what we have to work with. |
When you call Hive also seems to employ the HDFS trash:
I'm less sure what you mean on this one. We aren't making this change in the s3 file IO, but I'm less familiar with the differences between that and s3a. |
The issue is that the config can be different for the client than for the NameNode. So if a client configures
HadoopFileIO is an abstraction for all Hadoop FileSystem implementations (DistributedFileSystem, S3AFileSystem, GCSFileSystem, etc.). That means that if I enable this in |
Good point. Though, at the end of the day, I'm not sure that I see this differently from any other misconfiguration that an iceberg user might have that would adversely impact them. For example, we misconfigured a table location and then removed an entire hadoop directory thinking they were orphan files, haha!
Also a fair point. I think that I could resolve this one pretty safely using some instanceOf checks on the FileSystem object. Are you at all opposed to that? |
d0d62e8 to
e490367
Compare
As of now, the HadoopFileIO uses the Java delete API, which always skips using a configured trash directory. If the table's hadoop configuration has trash enabled, we should use it. We should also only do this for implementations where trashing files is acceptable. In our case, this is the LocalFileSystem and the DistributedFileSystem.
e490367 to
416c041
Compare
danielcweeks
left a comment
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.
@jordepic I think this looks good. When we get closer to the 1.11 release, please reach out to the release manager to highlight this in the release notes as it could have an impact on people running hdfs.
Thanks!
|
|
||
| private void deletePath(FileSystem fs, Path toDelete, boolean recursive) throws IOException { | ||
| Trash trash = new Trash(fs, getConf()); | ||
| if ((fs instanceof LocalFileSystem || fs instanceof DistributedFileSystem) |
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.
How about ViewFileSystem?
|
@jordepic @danielcweeks joining in very late here. that trash api really exists to stop users doing things on the command line, so I do see hive has it as a safety check, presumably someone did a DROP TABLE and changed their mind. I suspect it is not used on every file deletion though, more the whole-table operations, because one aspect of trash it likes to be atomic: moving a whole table in there gives you that. S3aFs doesn't like trash as the PoV there is "S3 versioning may not be atomic but it's a lot faster than renaming". We've discussed having a plugin policy here where each fs could have its own .HADOOP-18013. ABFS: add cloud trash policy with per-schema policy selection; superceded by something with active development apache/hadoop#8063 . I'll see about getting that in. Regarding this patch
What about just a configuration option "iceberg.hadoop.trash.schemas" to take a list of filesystem schemas "hdfs, viewfs, file, abfs" for which trash is enabled?. |
I like this idea as viewfs was not handled in this PR. @steveloughran Do you plan to open a follow-up PR? |
|
@manuzhang yes, I also want to get the bulk delete api calls in for cloud delete performance; the changes here are complicating that. I can do this change first. Then when ozone adds its own trash policy, it'll be easy to support |
|
Hi @steveloughran ! I'm sorry for the very late response on my end here. I'm happy to review or take care of the follow up change - let me know what you prefer. |
|
I'll have a go at the change |
As of now, the HadoopFileIO uses the Java delete API, which always skips using a configured trash directory. If the table's hadoop configuration has trash enabled, we should use it. We should also only do this for implementations where trashing files is acceptable. In our case, this is the LocalFileSystem and the DistributedFileSystem. Co-authored-by: Jordan Epstein <jordan.epstein@imc.com>
|
started the new PR. Also discovered a regression here, the moveToTrash() code raises an FNFE if there's no file/dir at the end of the path. This means that if ever a file which has already been deleted is deleted again: failure. Whereas filesystem.delete() is just a no-op. Going to catch an FNFE in moving a file. |
For HDFS the client actually asks the service what the policy is public FsServerDefaults getServerDefaults() throws IOException {
return this.dfs.getServerDefaults();
}ViewFS does this for the resolved FS of a path, so will get it for hdfs there. The PR #15111 uses this info so the entire trash settings should be picked up from the store. If a client config has trash off when working with local fs, s3, abfs etc, when it interacts with hdfs it'll still get the settings from that cluster. |
As of now, the HadoopFileIO uses the Java delete
API, which always skips using a configured trash
directory. If the table's hadoop configuration
has trash enabled, we should use it.