[ https://issues.apache.org/jira/browse/FLINK-11838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17283747#comment-17283747 ]
Galen Warren commented on FLINK-11838: -------------------------------------- Hi [~xintongsong], sorry for the delay here, I had some other, unrelated work I had to focus on. I like your idea of using {{WriteChannel}} for the uploads, but to close each one when {{RecoverableFsDataOutputStream.persist}} is called, so that we're not relying on {{WriteChannel}} for recoverability. {{WriteChannel}} allows one to control how frequently it flushes data via [setChunkSize|https://googleapis.dev/java/google-cloud-clients/0.90.0-alpha/com/google/cloud/WriteChannel.html#setChunkSize-int-], perhaps we expose this chunk size as a Flink option to give the user some control over the process, i.e. how much memory is used for buffering? It could be optional, not setting it would mean to use the Google default. Yes, it would be straightforward to compose blobs at any point in the process, i.e. on commit and/or at {{persist}} calls along the way. Maybe we compose them on commit (of course) and also whenever {{persist}} is called when there are at least 32 temp blobs to be composed? That way, we spread the compose work out over time but also call {{compose}} as few times as possible, composing as many blobs as possible in each call, which seems like it would be the most efficient. Shall we go with this approach? > Create RecoverableWriter for GCS > -------------------------------- > > Key: FLINK-11838 > URL: https://issues.apache.org/jira/browse/FLINK-11838 > Project: Flink > Issue Type: New Feature > Components: Connectors / FileSystem > Affects Versions: 1.8.0 > Reporter: Fokko Driesprong > Assignee: Galen Warren > Priority: Major > Labels: pull-request-available, usability > Fix For: 1.13.0 > > Time Spent: 20m > Remaining Estimate: 0h > > GCS supports the resumable upload which we can use to create a Recoverable > writer similar to the S3 implementation: > https://cloud.google.com/storage/docs/json_api/v1/how-tos/resumable-upload > After using the Hadoop compatible interface: > https://github.com/apache/flink/pull/7519 > We've noticed that the current implementation relies heavily on the renaming > of the files on the commit: > https://github.com/apache/flink/blob/master/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java#L233-L259 > This is suboptimal on an object store such as GCS. Therefore we would like to > implement a more GCS native RecoverableWriter -- This message was sent by Atlassian Jira (v8.3.4#803005)