[ 
https://issues.apache.org/jira/browse/FLINK-11838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17283950#comment-17283950
 ] 

Galen Warren edited comment on FLINK-11838 at 2/12/21, 9:01 PM:
----------------------------------------------------------------

One more thought/question. While we can potentially compose temp blobs along 
the way, to avoid having to compose them all at commit time, it seems to me 
that we can't safely delete any of the temporary blobs along way, because it's 
possible that we might restore to a checkpoint prior to some or all of the 
incremental compose operations having occurred. In that case, we'd have to 
repeat the compose operations, which means the underlying temp blobs would need 
to be there.

If that's right, then we'd necessarily have to wait to the end to delete the 
temp blobs. I was wondering, would it be allowable to do any of those delete 
operations in parallel? The coding 
[guidelines|https://flink.apache.org/contributing/code-style-and-quality-common.html#concurrency-and-threading]
 would seem to discourage this, but they don't outright prohibit it, so I 
thought I'd ask first.

EDIT: Nevermind regarding the parallelism question; I didn't notice that the 
GCS api provides a method to delete several blobs in a single call, I think 
that will suffice. 


was (Author: galenwarren):
One more thought/question. While we can potentially compose temp blobs along 
the way, to avoid having to compose them all at commit time, it seems to me 
that we can't safely delete any of the temporary blobs along way, because it's 
possible that we might restore to a checkpoint prior to some or all of the 
incremental compose operations having occurred. In that case, we'd have to 
repeat the compose operations, which means the underlying temp blobs would need 
to be there.

If that's right, then we'd necessarily have to wait to the end to delete the 
temp blobs. I was wondering, would it be allowable to do any of those delete 
operations in parallel? The coding 
[guidelines|https://flink.apache.org/contributing/code-style-and-quality-common.html#concurrency-and-threading]
 would seem to discourage this, but they don't outright prohibit it, so I 
thought I'd ask first.

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

Reply via email to