You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/12/19 10:37:37 UTC

[GitHub] [iceberg] zhongyujiang opened a new pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

zhongyujiang opened a new pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772


   I found that the _expire_snapshots_ and _remove_orphan_files_ may confilct when _remove_orphan_files_  procedure happens to start right after _expire_snapshots's_ commit and delete files before _expire_snapshots'_ delete, which will cause _expire_snapshots_ fails.
   This PR adds cleanExpiredFiles to Actions/ExpireSnapshots, which allows expiration of snapshots without any cleanup of files just like Iceberg/ExpireSnapshots. And the expired files will be left to _remove_orphan_files_.
   This PR only implent it in Spark3.2, so I convert it to draft.
   @aokolnychyi Do you think this is a reasonable way to avoid such conflict? Can you help review this pr? Thanks!
   
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997938486


   > @ajantha-bhat @rdblue Thanks for your review! Maybe I didn't make it clear, the failure of _expire_snapshots_ I met is not caused by deleting files instead of building expired files dataset, cause some files needed to build expiredFiles are deleted by _remove_orphan_files_ which happens to be started right after _expire_snapshots_' commit. So plugin dummy consumer method to avoid deleting the files has no effect on this situation. This situation is quite rare, but truely it happens. I think a simple way to avoid this is execute _remove_orphan_files_ after _expire_snapshots_ job is done, but our _remove_orphan_files_ procedure is automatically scheduled by program. So it's a bit difficult to control timing.
   
   I agree with Ryan, the thing to do here is to make it so that "expire snapshots" won't fail in this sort of situation. Basically just place in a bunch of try's to make it that if we can't determine the correct set of files to remove, we just log warnings for those files and continue as normal. This would result in not cleaning the total set of unused files since we wouldn't be able to determine the full set of unused files but since RemoveOrphans is running concurrently in this example, that really doesn't matter.
   
   That said the ExpireSnapshots method is far more efficient than RemoveOrphans and we should always be encouraging users towards using Expire to delete rather than Remove.For this particular use case I would probably just have the same program that runs expire snapshots also run remove orphans so that they never run concurrently. 


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772304946



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       ```
   Dataset<Row> originalFiles = buildValidFileDF(ops.current());
   ```
   The originalFiles dataset is calculated lazily, so the situation is like this:
   expireSnapshots.commit() -> originalFiles.except(validFiles) (ongoing) -> _remove_orphan_files_ start and delete some file for building _originalFiles_ -> _expire_snapshots_ falis baceuse of FileNotFoundException.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772378889



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       I agree, we already have APIs for expiring without deleting and IMHO we should be pushing users towards deleting with expire snapshots. So anything that makes it seem like this is not the right way to clean old files should be avoided. 




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772304946



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       ```
   Dataset<Row> originalFiles = buildValidFileDF(ops.current());
   ```
   The originalFiles dataset is calculated lazily, so the situation is like this:
   expireSnapshots.commit() -> originalFiles.except(validFiles) (calculating) -> _remove_orphan_files_ start and delete some file for building _originalFiles_ -> _expire_snapshots_ falis baceuse of FileNotFoundException.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772370782



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       Intersting. 
   I am rewording for my understanding...
   
   `expireSnapshots.commit()` has removed snapshot entries from metadata. 
   But the files (manifest list or manifest files) referring from that snapshot was present in `originalFiles` dataframe.
   So, when `remove_orphan_files` called during that time, as metadata entry doesn't exist for those files, it has deleted it. 
   **So, when  `originalFiles.except(validFiles)` is called to actually compute the `originalFiles`, the manifest list or manifest files will be read which was deleted by `remove_orphan_files`. Leading to `FileNotFoundException`.**
   
   In your PR, `expireSnapshots` will remove the expired snapshot entires from the metadata but doesn't return list of removed files (which can potentially cause FileNotFoundException in above scenario)  and also doesn't delete the files as output list is empty. 
   And these files will be later cleaned by running `remove_orphan_files` again. [Also note that cleaning expired files from `remove_orphan_files` is costlier (slower) compared to cleaning them in expiredSnapshots.
   
   This PR can solve the issue. In my opinion, it looks hacky and user need to know when to set configuration to skipClean and when not to set it. I suggest we can block `remove_orphan_files` when `expireSnapshots` is running instead and vice versa (using event listener or some lock?). We can wait for others opinion.
   cc: @rdblue , @RussellSpitzer 
   
   
   
   
   
   




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772431408



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       Your understanding is quite right. But I don't think event listener or lock is a good idea, cause that seems need to introduce external dependencies.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772373957



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       or one more way to fix this is to just skip deleting the manifest file and manifest list but still delete the data files. But it is also hacky. 




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772304946



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       The situation is like this:
   expireSnapshots.commit() -> originalFiles.except(validFiles) (ongoing) -> _remove_orphan_files_ start and delete some file for building _originalFiles_ -> _expire_snapshots_ falis baceuse of FileNotFoundException.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang edited a comment on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang edited a comment on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997996662


   > That said the ExpireSnapshots method is far more efficient than RemoveOrphans and we should always be encouraging users towards using Expire to delete rather than Remove.
   
   This make sense to me, how about just trigger calculation for **Dataset<Row> originalFiles** before commit and cache it to reduce the probability of such failure.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997856672


   @ajantha-bhat @rdblue Thanks for your review! Maybe I didn't make it clear, the failure of _expire_snapshots_ I met is not caused by deleting files instead of building expired files dataset, cause some files needed to build expiredFiles are deleted by _remove_orphan_files_ which happens to be started right after _expire_snapshots_' commit. So plugin dummy consumer method to avoid deleting the files has no effect on this situation.
   This situation is quite rare, but truely it happens. I think a simple way to avoid this is execute  _remove_orphan_files_ after _expire_snapshots_ job is done, but our _remove_orphan_files_ procedure is automatically scheduled by program. So it's a bit difficult to control timing.
   
   
   
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772370782



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       Intersting. 
   I am rewording for my understanding...
   
   `expireSnapshots.commit()` has removed snapshot entries from metadata. 
   But the files (manifest list or manifest files) referring from that snapshot was present in `originalFiles` dataframe.
   So, when `remove_orphan_files` called during that time, as metadata entry doesn't exist for those files, it has deleted it. 
   So, when  `originalFiles.except(validFiles)` is called to actually compute the `originalFiles`, the manifest list or manifest files will be read which was deleted by `remove_orphan_files`. Leading to `FileNotFoundException`.
   
   In your PR, `expireSnapshots` will remove the expired snapshot entires from the metadata but doesn't return list of removed files (which can potentially cause FileNotFoundException in above scenario)  and also doesn't delete the files as output list is empty. 
   And these files will be later cleaned by running `remove_orphan_files` again. [Also note that cleaning expired files from `remove_orphan_files` is costlier (slower) compared to cleaning them in expiredSnapshots.
   
   This PR can solve the issue. In my opinion, it looks hacky and user need to know when to set configuration to skipClean and when not to set it. I suggest we can block `remove_orphan_files` when `expireSnapshots` is running instead and vice versa (using event listener or some lock?). We can wait for others opinion.
   cc: @rdblue , @RussellSpitzer 
   
   
   
   
   
   




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-998170327


   @zhongyujiang, caching is an optimization and we should not rely on it for correctness. There is no guarantee that cached data will still be available. Plus, it is expensive and causes scaling issues in Spark clusters.
   
   Instead, I think that if a metadata file is missing, we should ignore it and move on. This should warn in that case but otherwise be a best effort delete.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997996662


   > That said the ExpireSnapshots method is far more efficient than RemoveOrphans and we should always be encouraging users towards using Expire to delete rather than Remove.
   
   This make sense to me, how about just trigger calculation for **Dataset<Row> originalFiles** before commit and cache it.
   
   
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997440123


   I agree with @ajantha-bhat that there's already a way to do this. I don't think we should expose this to users, who probably would not understand the situations in which you'd want to not clean up data files.
   
   The situation you described seems to be a different bug to me. I don't think that deleting data files should ever cause the job to fail. We should catch errors and log them, but continue trying to clean up.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang edited a comment on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang edited a comment on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997996662


   > That said the ExpireSnapshots method is far more efficient than RemoveOrphans and we should always be encouraging users towards using Expire to delete rather than Remove.
   
   This makes sense, I think we can trigger calculation for **Dataset<Row> originalFiles** before commit and cache it to reduce the probability of such failure. @RussellSpitzer What do you think? cc @ajantha-bhat 


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r772489476



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java
##########
@@ -172,11 +179,14 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc)
 
       expireSnapshots.commit();

Review comment:
       Agree. Your new solution of "trigger calculation for Dataset originalFiles before commit and cache it to reduce the probability of such failure" sounds good to me 👍🏻




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r771983427



##########
File path: api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java
##########
@@ -92,6 +92,16 @@
    */
   ExpireSnapshots executeDeleteWith(ExecutorService executorService);
 
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * If {@code cleanEnabled} is set to false, data and manifest files should not been cleaned after expiration.
+   *
+   * @param cleanEnabled setting this to false will skip deleting expired manifests and files
+   * @return this for method chaining
+   */
+  ExpireSnapshots cleanExpiredFiles(boolean cleanEnabled);

Review comment:
       This action is already having `deleteWith(Consumer<String> deleteFunc);` method. 
   **we can plugin dummy consumer method to avoid deleting the files.**
   
   Default implementation is to delete files. 
   https://github.com/apache/iceberg/blob/master/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java#L84-L89
   
   So, I think this PR is not required.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r771985075



##########
File path: api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java
##########
@@ -92,6 +92,16 @@
    */
   ExpireSnapshots executeDeleteWith(ExecutorService executorService);
 
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * If {@code cleanEnabled} is set to false, data and manifest files should not been cleaned after expiration.
+   *
+   * @param cleanEnabled setting this to false will skip deleting expired manifests and files
+   * @return this for method chaining
+   */
+  ExpireSnapshots cleanExpiredFiles(boolean cleanEnabled);

Review comment:
       But we can have changes for call procedure I guess. We can expose an argument to `skipDelete` and use the dummy consumer in spark actions of this call procedure if the `skipDelete` = true.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ajantha-bhat commented on a change in pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#discussion_r771983427



##########
File path: api/src/main/java/org/apache/iceberg/actions/ExpireSnapshots.java
##########
@@ -92,6 +92,16 @@
    */
   ExpireSnapshots executeDeleteWith(ExecutorService executorService);
 
+  /**
+   * Allows expiration of snapshots without any cleanup of underlying manifest or data files.
+   * <p>
+   * If {@code cleanEnabled} is set to false, data and manifest files should not been cleaned after expiration.
+   *
+   * @param cleanEnabled setting this to false will skip deleting expired manifests and files
+   * @return this for method chaining
+   */
+  ExpireSnapshots cleanExpiredFiles(boolean cleanEnabled);

Review comment:
       This action is already having `deleteWith(Consumer<String> deleteFunc);` method. 
   **we can plugin dummy consumer method to avoid deleting the files.**
   
   Default implementation is to delete files. 
   https://github.com/apache/iceberg/blob/master/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java#L84-L89




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772#issuecomment-997440548


   @zhongyujiang can you give more detail on what failed? Looking at the code that deletes files, I think that it should already log a warning and skip the file delete, rather than fail. https://github.com/apache/iceberg/blob/master/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseExpireSnapshotsSparkAction.java#L245-L271
   
   What failure did you see?


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhongyujiang closed pull request #3772: [Api][Spark]Add cleanExpiredFiles to Actions/ExpireSnapshots

Posted by GitBox <gi...@apache.org>.
zhongyujiang closed pull request #3772:
URL: https://github.com/apache/iceberg/pull/3772


   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org