sririshindra opened a new pull request, #7127:
URL: https://github.com/apache/iceberg/pull/7127
### What changes are proposed in this PR?
---
This PR adds a new Interface OrphanFileStatus that indicates if an orphan
file was deleted or not. In cases of failure during file deletion, It provides
a reference to the encountered exception.
With this additional information, a user can choose to record this failure
and retry the deletion process in a controlled fashion in the future.
### Why are the changes are needed?
During the execution of the DeleteOrphanFiles spark action or
remove_orphan_files SQL procedure, a failure encountered during deletion of the
orphan file is not bubbled up to the user nor is there any indication of the
failure in the returned result.
The return value of the Spark action is Iterable and the SQL procedure
simply displays a list of orphan files in a table. With this limited
information, the only way for the user to know if the orphan file was deleted
or not is to
- grep the logs for the warning
- for each of the locations returned in the iterable, query the cloud
storage for its existence
One can re-run the expensive delete action to delete the files that failed
during the previous delete action run, however, that is not very desirable
since re-listing the dir contents to identify the orphan files would result in
duplicate work plus wastage of precious API calls. One of the common causes of
delete failure in the public cloud is hitting the API quotas and unnecessary
re-runs of the delete action can lead to a temporary denial of access to other
workloads accessing the same storage resource.
Providing information about the failed delete files gives users the ability
to take appropriate action based on their use case.
## Output
### Before this change
#### DryRun => true/false
```
scala> sql("CALL spark_catalog.system.remove_orphan_files(table => 't1',
older_than => TIMESTAMP '2022-05-21 00:00:00.000', dry_run =>
true)").show(false)
+--------------------------------------------------------------------------------------------------------------------------+
|orphan_file_location
|
+--------------------------------------------------------------------------------------------------------------------------+
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00000-705370fd-ea3e-4ae7-8b40-c0362b0ba7df-c000.snappy.parquet|
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00001-705370fd-ea3e-4ae7-8b40-c0362b0ba7df-c000.snappy.parquet|
+--------------------------------------------------------------------------------------------------------------------------+
```
### After this change
#### DryRun => true
```
scala> sql("CALL spark_catalog.system.remove_orphan_files(table => 't1',
older_than => TIMESTAMP '2022-05-21 00:00:00.000', dry_run =>
true)").show(false)
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
|orphan_file_location
|deleted|error_message|
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00000-8251ec5f-dd9b-4754-8e68-2cdd01e66e56-c000.snappy.parquet|false
|null |
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00001-8251ec5f-dd9b-4754-8e68-2cdd01e66e56-c000.snappy.parquet|false
|null |
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
```
#### DryRun => false
```
scala> sql("CALL spark_catalog.system.remove_orphan_files(table => 't1',
older_than => TIMESTAMP '2022-05-21 00:00:00.000', dry_run =>
false)").show(false)
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
|orphan_file_location
|deleted|error_message|
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00000-8251ec5f-dd9b-4754-8e68-2cdd01e66e56-c000.snappy.parquet|true
|null |
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00001-8251ec5f-dd9b-4754-8e68-2cdd01e66e56-c000.snappy.parquet|true
|null |
+--------------------------------------------------------------------------------------------------------------------------+-------+-------------+
```
#### DryRun => false, simulating failure during delete
```
+--------------------------------------------------------------------------------------------------------------------------+-------+---------------------------------------+
|orphan_file_location
|deleted|error_message
|
+--------------------------------------------------------------------------------------------------------------------------+-------+---------------------------------------+
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00000-615125af-bfbb-4279-b22b-559dee4f0b13-c000.snappy.parquet|true
|null |
|file:/tmp/iceberg_warehouse/default/t1/non_table_files/part-00001-705370fd-ea3e-4ae7-8b40-c0362b0ba7df-c000.snappy.parquet|false
|simulating failure during file deletion|
+--------------------------------------------------------------------------------------------------------------------------+-------+---------------------------------------+
```
## Code Credit
This PR is a reimplementation of #4817 published by @sumeetgajjar for
spark3.2. As @sumeetgajjar mentioned in #4817 he moved on to other projects and
I am taking this up. I tried cherry-picking Sumeet's changes but I had a hard
time resolving the conflicts, It was easier to reimplement the change on master
directly. I am not sure how to give credit to Sumeet's work. @sumeetgajjar
maybe you can push a dummy commit on top of this branch so that if and when
this PR gets merged, it will show your name as well in the commit message.
In the original PR(#4817) @RussellSpitzer and @aokolnychyi said that they
can take a look at it after
[#4652](https://github.com/apache/iceberg/pull/4652) is merged. @RussellSpitzer
@aokolnychyi @szehon-ho Could you please take a look at this PR as
[#4652](https://github.com/apache/iceberg/pull/4652) is now merged. Apart from
the changes @sumeetgajjar did in #4817 I implemented this feature for bulk
deletes as well. I also implemented it for spark3.3 rather than spark3.2. If
this changes goes in I can raise a separate PRs for spark3.2 and spark3.1 as
well.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]