I merged the PR to revert this since I don't think anyone is strongly for
keeping it. I also think Steve is right that if we have NN pressure we
would want to use a bulk endpoint and that it won't be better to use
renames.

The original author also confirmed on the PR that they can use a custom
FileIO for this and don't need it to be in Iceberg. That use case was
around having a way to undo bad orphan file cleanup, which would delete
files underneath the table. I don't think that's really an Iceberg
responsibility, again because if it were it would be built into the Hadoop
FileSystem rather than the FileIO layer above.

It would also be a good idea to think about how to alternatively address
those cases. I think replicas are a good way to address it that are going
to be easier to produce in v4 (with relative paths) but other ideas are
definitely welcome here!

On Mon, Feb 23, 2026 at 3:07 AM Steve Loughran <[email protected]> wrote:

>
> On Sat, 21 Feb 2026 at 09:02, Cheng Pan <[email protected]> wrote:
>
>> Share a use case of HDFS Trash - deleting a directory on HDFS that has
>> tons of files might cause significant pressure on the NameNode and
>> slow the HDFS cluster for dozens of minutes, while moving to Trash is
>> relatively cheap, then those files can be deleted in the background
>> after reaching expiration time, in small batches, thus no pressure and
>> latency on the NameNode.
>>
>>
> iceberg is only to be deleting files though, not directories; it''ll be
> acquiring a lock per file for a delete, and for a rename needs to get a
> lock of ~.Trash too. I don't see it being any worse here.
>
> Now, if you were to add bulk delete support to hdfs, we could send a
> single RPC there with a batch of files and hdfs could go through them and
> delete in turn, failing if a dir was encountered.And like the s3a
> implementation, it could be throttled: you'd implement that on the server
> before actually acquiring any locks so all callers of bulk delete would be
> constrained
>
>
>
>> If possible, I would still like Iceberg to have this feature.
>>
>> Thanks,
>> Cheng Pan
>>
>> On Fri, Feb 20, 2026 at 3:22 AM Daniel Weeks <[email protected]> wrote:
>> >
>> > I agree with Steve and Ryan on this.
>> >
>> > I was a bit critical of all the issues with configuration and behavior
>> when reviewing the PR, but felt that containing it to HDFS might make it
>> reasonable to close the gap in behavior between Hive tables and Iceberg.
>> >
>> > However, it is complicated, messy and could cause surprising behavior
>> for anyone who has it turned on in their environment when it suddenly
>> starts being respected causing lots of trash behavior.
>> >
>> > I'll open a PR to revert and reach out to the original author.
>> >
>> > -Dan
>> >
>> > On Thu, Feb 19, 2026 at 11:14 AM Steve Loughran <[email protected]>
>> wrote:
>> >>
>> >>
>> >> I'm very happy with removing support; it just complicates the code for
>> a failure condition "accidental deletion" which shouldn't surface.
>> >>
>> >> The only times where the users may want to roll back a delete is DROP
>> TABLE, and there it's the homework of the catalog to give users a way to
>> revert it.
>> >>
>> >> It's not shipped yet so removal is not a regression at all.
>> >>
>> >> steve
>> >>
>> >>
>> >> On Wed, 18 Feb 2026 at 22:48, Ryan Blue <[email protected]> wrote:
>> >>>
>> >>> During the Iceberg sync this morning, Steve suggested a PR to fix a
>> problem with HadoopFileIO, #15111. I looked into this a bit more and it is
>> based on #14501, which implements a Hadoop scheme where delete may actually
>> move a file to a configured trash directory rather than deleting it. I
>> think that this trash behavior is strange and doesn't fit into FileIO. I
>> think the right thing to do is to probably remove it but I want to see what
>> arguments for the behavior there are.
>> >>>
>> >>> In my opinion, the trash behavior is confusing and not obvious for
>> the FileIO interface. The behavior, as I understand it, is to check whether
>> a file should actually be deleted or should just be moved to a trash
>> folder. Interestingly, this is not done underneath the Hadoop FileSystem
>> interface, but is a client responsibility. Since FileIO is similar to
>> FileSystem, I think there's a strong argument that it isn't appropriate
>> within FileIO either. But there's another argument for not having this
>> behavior, which is that table changes and user-driven file changes are not
>> the same. Table can churn files quite a bit and deletes shouldn't move
>> uncommitted files to trash -- they don't need to be recovered -- nor should
>> they move replaced or deleted data files to a trash folder that could be in
>> a user's home directory -- this is a big and not obvious behavior change.
>> This seems to be in conflict with reasonable governance schemes because it
>> could leak sensitive data.
>> >>>
>> >>> Next, the use case for a trash folder is to recover from accidental
>> deletes by users. This is unnecessary in Iceberg because tables keep their
>> own history. Accidental data operations are easily rolled back and we have
>> a configurable history in which you can do it. This is also already
>> integrated cleanly so that temporary metadata files that end up not being
>> committed are not held.
>> >>>
>> >>> In the end, I think that we don't need this because history is
>> already kept in a better way for tables, and this feature is confusing and
>> doesn't fit in the API. What are the use cases for keeping this?
>> >>>
>> >>> Ryan
>>
>

Reply via email to