[ https://issues.apache.org/jira/browse/FLINK-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16589854#comment-16589854 ]
ASF GitHub Bot commented on FLINK-9061: --------------------------------------- StefanRRichter commented on a change in pull request #6604: [FLINK-9061] Optionally add entropy to checkpoint paths better S3 scalability URL: https://github.com/apache/flink/pull/6604#discussion_r212211288 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java ########## @@ -63,6 +78,7 @@ public FsCheckpointStorage( this.sharedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_SHARED_STATE_DIR); this.taskOwnedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_TASK_OWNED_STATE_DIR); this.fileSizeThreshold = fileSizeThreshold; + this.pathFilter = checkNotNull(pathFilter); Review comment: Isn't this problematic here: if we have a concept that paths can contain an entropy key, we still call `mkdirs` on the "unfiltered" paths in the following lines. It seems to clash a bit with the storage location idea. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > add entropy to s3 path for better scalability > --------------------------------------------- > > 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.5.0, 1.4.2 > Reporter: Jamie Grier > Assignee: Indrajit Roychoudhury > Priority: Critical > Labels: pull-request-available > > 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)