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

Stephan Ewen commented on FLINK-9061:
-------------------------------------

That would be a great contribution, valuable for many S3 users. Indeed, S3's 
partitioning by prefix 

FYI: The 1.5 release will have a few changes to the state backends that will 
reduce some S3 requests (mainly HEAD, if you use s3a).
The here proposed is still needed, though.

We could approach this in two ways:

  1. Make the change to the state backend itself, to allow it to insert entropy 
into the paths. We can think about only doing this for checkpoints, but not for 
savepoints, because then savepoints would still be together in "one directory" 
(under the same prefix).

  2. Make this a general change in the S3 filesystem, to use a deterministic 
path hash like Jamie suggested. That would help to solve this across other use 
cases of S3 as well. Whenever the path contains the segment *_hash_* this would 
be replaced by such a hash.

> S3 checkpoint data not partitioned well -- causes errors and poor performance
> -----------------------------------------------------------------------------
>
>                 Key: FLINK-9061
>                 URL: https://issues.apache.org/jira/browse/FLINK-9061
>             Project: Flink
>          Issue Type: Bug
>          Components: FileSystem, State Backends, Checkpointing
>    Affects Versions: 1.4.2
>            Reporter: Jamie Grier
>            Priority: Critical
>
> I think we need to modify the way we write checkpoints to S3 for high-scale 
> jobs (those with many total tasks).  The issue is that we are writing all the 
> checkpoint data under a common key prefix.  This is the worst case scenario 
> for S3 performance since the key is used as a partition key.
>  
> In the worst case checkpoints fail with a 500 status code coming back from S3 
> and an internal error type of TooBusyException.
>  
> One possible solution would be to add a hook in the Flink filesystem code 
> that allows me to "rewrite" paths.  For example say I have the checkpoint 
> directory set to:
>  
> s3://bucket/flink/checkpoints
>  
> I would hook that and rewrite that path to:
>  
> s3://bucket/[HASH]/flink/checkpoints, where HASH is the hash of the original 
> path
>  
> This would distribute the checkpoint write load around the S3 cluster evenly.
>  
> For reference: 
> https://aws.amazon.com/premiumsupport/knowledge-center/s3-bucket-performance-improve/
>  
> Any other people hit this issue?  Any other ideas for solutions?  This is a 
> pretty serious problem for people trying to checkpoint to S3.
>  
> -Jamie
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to