[ https://issues.apache.org/jira/browse/FLINK-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16544603#comment-16544603 ]
ASF GitHub Bot commented on FLINK-9061: --------------------------------------- Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/6302 Thanks for this contribution, that's a valuable fix. I have a few thoughts and suggestions on how we might improve the feature a bit still: - Can we get id of the `commons-text` dependency? The fewer dependencies, the fewer possible problems for users due to dependency clashes. It seems a bit heavy to add a new library for just one random string generation. - The feature is configured through additional constructor parameters. I am wondering if we may want to move this to the `Configuration`. That would allow the "ops side of things" to configure this for a setup (setting entropy key and checkpoints directory) without needing everyone that writes a Flink program to be aware of this. - If I read the code correctly, the code logs warnings for every file in case the feature is not activated. That will probably confuse a lot of users and make them dig into whether they have a wrong setup, when they simply don't use this new feature. > 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)