galenwarren opened a new pull request #14875:
URL: https://github.com/apache/flink/pull/14875


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that 
you want to help us improve Flink. To help the community review your 
contribution in the best possible way, please go through the checklist below, 
which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a 
hassle. In order to uphold a high standard of quality for code contributions, 
while at the same time managing a large number of contributions, we need 
contributors to prepare the contributions well, and give reviewers enough 
contextual information for the review. Please also understand that 
contributions that do not follow this guide will take longer to review and thus 
typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA 
issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are 
made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the 
pull request", where *FLINK-XXXX* should be replaced by the actual issue 
number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following 
this pattern: `[hotfix] [docs] Fix typo in event time introduction` or 
`[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the 
pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean 
verify` passes. You can set up Azure Pipelines CI to do that following [this 
guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from 
multiple issues.
     
     - Each commit in the pull request has a meaningful commit message 
(including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and 
this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   Add a FileSystem implementation with associated RecoverableWriter 
implementation for Google Cloud Storage, in order to be able to write to GCS 
via a StreamingFileSink. 
   
   ## Brief change log
   - Add GSFileSystemFactory and register with SPI under 
org.apache.flink.core.fs.FileSystemFactory. This factory supports the ```gs``` 
scheme.
   - GSFileSystemFactory creates instances of GSFileSystem, which extends 
HadoopFileSystem. An instance of [GoogleHadoopFileSystem 
](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/src/main/java/com/google/cloud/hadoop/fs/gcs/GoogleHadoopFileSystem.java)
 (i.e. the standard Hadoop connector for GCS) is used to construct 
HadoopFileSystem.
   - GSFileSystem creates instances of GSRecoverableWriter via 
```createRecoverableWriter```
   - GSRecoverableWriter writes files to GCS in a two-step process, by first 
writing to a temporary blob, using the [resumable 
upload](https://cloud.google.com/storage/docs/resumable-uploads) feature, and 
then copying the temp blob to the final blob on commit (and deleting the temp 
file). Resumable uploads can be active for up to one week.
   - Recoverability during the resumable upload process is provided by the 
```capture``` method on 
[WriteChannel](https://googleapis.dev/java/google-cloud-clients/0.90.0-alpha/com/google/cloud/WriteChannel.html)
 and the ```restore``` method on 
[RestorableState<WriteChannel>](https://googleapis.dev/java/google-cloud-clients/0.90.0-alpha/com/google/cloud/RestorableState.html).
 GSRecoverable holds the recoverable state, which includes:
     - The opaque RestorableState object returned from 
[WriteChannel.capture](https://googleapis.dev/java/google-cloud-clients/0.90.0-alpha/com/google/cloud/WriteChannel.html#capture--)
 (which is Java serializable)
     - The position in the write stream
     - The "plan", i.e. an instance of GSRecoverablePlan, which includes the 
temporary and final blob ids, to be used for eventual commit or cleanup
    
   Four new Flink options are added:
   - **gs.upload.content.type**. This is the content type to use when writing 
blobs. Default: ```application/octet-stream```.
   - **gs.upload.temp.bucket**. The bucket to use for temporary blobs. There is 
no default; if unspecified, temporary blobs are written to the same bucket as 
the final blobs. Specifying this option would allow one to put temporary blobs 
in their own bucket.
   - **gs.upload.temp.prefix**. The prefix to apply to the final object name to 
generate the temporary object name. A UUID string is also appended to the 
temporary object name to avoid collisions during the resumable upload process. 
Default value: ```.inprogress```
   - **gs.upload.chunk.size**. The chunk size to set for uploads. If zero (the 
default), no chunk size is set and the Google default value is used.
   
   So, for example -- if a write were initiated to 
```gs://bucket-name/foo/bar```, the following would happen (assuming default 
values for the options above):
   - A temporary blob would be created at, say, 
```gs://bucket-name/.inprogress/foo/bar/e202b566-150f-4784-9025-136cf351fcfe```
   - Writes would be performed against this blob via resumable upload for up to 
a week, with support for saving and restoring the upload state
   - Upon commit, the temporary blob would be copied to 
```gs://bucket-name/foo/bar``` and the temporary blob would be deleted
   - Upon failure, the temporary blob would be deleted (via 
[RecoverableWriter.cleanupRecoverableState](https://ci.apache.org/projects/flink/flink-docs-master/api/java/org/apache/flink/core/fs/RecoverableWriter.html#cleanupRecoverableState-org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable-))
   
   A few important implementation notes:
   - The GSRecoverableWriterHelper class and its child interfaces -- Storage, 
Writer, and WriterState -- abstract away the interactions with the underlying 
GCS interfaces. DefaultGSRecoverableWriterHelper implements 
GSRecoverableWriterHelper against the underlying GCS interfaces; for 
unit-testing purposes, MockGSRecoverableWriterHelper implements 
GSRecoverableWriterHelper against in-memory objects.
   - GSRecoverableWriterHelper.WriterState implements Serializable, as it wraps 
an instance of the opaque 
[RestorableState<WriteChannel>](https://googleapis.dev/java/google-cloud-clients/0.90.0-alpha/com/google/cloud/RestorableState.html)
 returned from the GCS API. RestorableState instances are required to be Java 
serializable, and this would seem to be the only way to serialize/deserialze 
them. **So, GSRecoverableSerializer uses Java serialization for WriteState**. 
I'm aware this goes against the guidance not to use Java serialization, but, in 
this case, I don't see an alternative (but I'm open to suggestions).
   
   ## Verifying this change
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe 
tests)*.
   
   *(or)*
   
   **This change added tests and can be verified as follows**:
   - Added unit tests in GSFileSystem test to verify the proper function of 
GSFileSystemFactory and GSFileSystem, including Flink options and passthrough 
of options to Hadoop. No validation of the external 
[GoogleHadoopFileSystem](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/src/main/java/com/google/cloud/hadoop/fs/gcs/GoogleHadoopFileSystem.java)
 is performed.
   - Added unit tests in GSRecoverableWriterTest to test scenarios related to 
writing, persist/restore, and cleanup.
   - Added unit tests in GSRecoverableSerializerTest to test serialization of 
GSRecoverable.
   - Added unit tests in GSRecoverableCommitterTest to test commit scenarios, 
including commit after recovery.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (**yes** / no)
       - Adds a new project -- **flink-gs-fs-hadoop** -- which includes 
dependencies 
[google-cloud-storage](https://mvnrepository.com/artifact/com.google.cloud/google-cloud-storage)
 and 
[gcs-connector](https://mvnrepository.com/artifact/com.google.cloud.bigdataoss/gcs-connector).
 I have a question about the proper scope for gcs-connector which I'll ask in a 
follow-up comment.
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
     - The serializers: (**yes** / no / don't know)
       - Adds GSRecoverableSerializer, to serialize instances of GSRecoverable
     - The runtime per-record code paths (performance sensitive): (yes / **no** 
/ don't know)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / 
don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / **docs** / 
**JavaDocs** / not documented)
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to