Skip to content

Conversation

@scwhittle
Copy link
Contributor

@scwhittle scwhittle commented Aug 10, 2021

Previously exceptions such as timeouts or unavailable for a single file would not be caught
and would cause retries and backoff of the entire DoFn processing.

Additionally plumb the options to ignore missing sources through to underlying FileSystem.rename
so that file systems can handle such errors intelligently per-file. The filtering in FileSystems.rename
to remove such files is beneficial because it may use cheaper match operations but it is insufficient
as previous deletes observed as timeouts may delete files between the match and rename.

Further reduce operations by modifying FileBasedSink to not delete files that have been renamed.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Format the pull request title like [BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replace BEAM-XXX with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

ValidatesRunner compliance status (on master branch)

Lang ULR Dataflow Flink Samza Spark Twister2
Go --- Build Status Build Status Build Status Build Status ---
Java Build Status Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Build Status
Python --- Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status ---
XLang Build Status Build Status Build Status Build Status Build Status ---

Examples testing status on various runners

Lang ULR Dataflow Flink Samza Spark Twister2
Go --- --- --- --- --- --- ---
Java --- Build Status
Build Status
Build Status
--- --- --- --- ---
Python --- --- --- --- --- --- ---
XLang --- --- --- --- --- --- ---

Post-Commit SDK/Transform Integration Tests Status (on master branch)

Go Java Python
Build Status Build Status Build Status
Build Status
Build Status

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website Whitespace Typescript
Non-portable Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status Build Status Build Status
Portable --- Build Status Build Status --- --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests

See CI.md for more information about GitHub Actions CI.

Copy link
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we still need removeTemporaryFiles() to remove the temporary directory for batch.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

adding back, I think we need this as well if we are filtering if the destination exists because we still need to remove the source in that case.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Might make sense to update the semantics of "StandardMoveOptions.SKIP_IF_DESTINATION_EXISTS" to delete the source during rename. That will allow us to prevent the double delete for the case where the source existed.

Also probably removeTemporaryFiles() should be updated to just cleanup the temporary directory (where appropriate) instead of trying to delete already renamed files.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I modified FileSystems.rename to delete srcs that existed but were filtered due to dest existing.
I kept the existing methods in FileBasedSink because it appears they are designed to be called by subclasses. I changed to pass an empty set for known files after the rename to avoid the unnecessary delete.

@scwhittle scwhittle force-pushed the gcs_errors branch 2 times, most recently from 8d30e0a to e288ade Compare August 11, 2021 13:19
@scwhittle scwhittle changed the title Change renames to handle retries internally [BEAM-12740] [BEAM-8268] Improve error handling and retries for GCS rename used by FileBasedSink Aug 11, 2021
@scwhittle
Copy link
Contributor Author

PTAL, I removed the hacks for testing since the performance seemed greatly improved.
Additionally I removed the unnecessary delete of renamed files in FileBasedSink by improving FileSystems.remove to always delete src files (even if the copy is skipped due to dest existing and that being filtered).
I added a unit test for rename by adding support to inject a batch object. The apiary BatchRequest object is difficult to mock so I opted for mocking above that.

@scwhittle
Copy link
Contributor Author

@reuvenlax @chamikaramj

Copy link
Contributor

@chamikaramj chamikaramj left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. LGTM other than one comment.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should raise an UnsupportedOperationexception for this and other FileSystems when MoveOptions that are not fully supported are used.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done, changed FileSystems to catch the exception and retry without move options to keep existing behavior for filesystems that don't support options.

I also fixed bug where I wasn't forwarding on the move options as I thought to the filesystem in FileSystems.rename ( and thus wasn't using the improved error handling in GcsFileSystem)

@chamikaramj
Copy link
Contributor

Run Java PreCommit

@chamikaramj
Copy link
Contributor

Run Java_Examples_Dataflow PreCommit

@chamikaramj
Copy link
Contributor

Run Java_Examples_Dataflow_Java11 PreCommit

@chamikaramj
Copy link
Contributor

Not sure why word count failed, but re-trying filed tests.

…ename operations

Previously exceptions such as timeouts or unavailable for a single file would not be caught
and would cause retries and backoff of the entire DoFn processing.

Additionally plumb the options to ignore missing sources through to underlying FileSystem.rename
so that file systems can handle such errors intelligently per-file.  The filtering in FileSystems.rename
to remove such files is beneficial because it may use cheaper match operations but it is insufficient
as previous deletes observed as timeouts may delete files between the match and rename.

Further reduce operations by modifying FileBasedSink to not delete files that have been renamed.
@chamikaramj
Copy link
Contributor

LGTM. Thanks.

@chamikaramj
Copy link
Contributor

Run Java PreCommit

1 similar comment
@scwhittle
Copy link
Contributor Author

Run Java PreCommit

@scwhittle
Copy link
Contributor Author

Run Java_Examples_Dataflow PreCommit

@scwhittle
Copy link
Contributor Author

Run Java_Examples_Dataflow_Java11 PreCommit

@scwhittle
Copy link
Contributor Author

Run Java PreCommit

@scwhittle
Copy link
Contributor Author

Run Java_Examples_Dataflow PreCommit

@scwhittle
Copy link
Contributor Author

Run Java_Examples_Dataflow_Java11 PreCommit

@lukecwik
Copy link
Member

Run Java_Examples_Dataflow PreCommit

@lukecwik
Copy link
Member

Run Java_Examples_Dataflow_Java11 PreCommit

@scwhittle
Copy link
Contributor Author

@chamikaramj @lukecwik Tests are passing now, I think this is ready to merge

@chamikaramj
Copy link
Contributor

Thanks. Yeah, we can merge.

@chamikaramj chamikaramj merged commit a0fbe00 into apache:master Aug 18, 2021
@scwhittle scwhittle deleted the gcs_errors branch August 30, 2021 09:58
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants