[
https://issues.apache.org/jira/browse/FLINK-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16539526#comment-16539526
]
ASF GitHub Bot commented on FLINK-9061:
---------------------------------------
GitHub user indrc opened a pull request:
https://github.com/apache/flink/pull/6302
[FLINK-9061][checkpointing] add entropy to s3 path for better scalability
## What is the purpose of the change
*This pull request adds entropy to the checkpoint path based upon a user
defined pattern in the configuration for better s3 scalability*
## Brief change log
- *Read the entropy pattern from the config*
- *In the checkpoint path for FSbackend, substitute the entropy key with
a 4 character random alpha numeric string*
## Verifying this change
This change added tests and can be verified as follows:
- *Added unit tests under FileStateBackendTest to cover different
scenarios for the entropy substitution*
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): (yes )
- The public API, i.e., is any changed class annotated with
`@Public(Evolving)`: ( no)
- The serializers: ( no )
- The runtime per-record code paths (performance sensitive): (no)
- Anything that affects deployment or recovery: JobManager (and its
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes )
- The S3 file system connector: (no)
## Documentation
- Does this pull request introduce a new feature? (no)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/indrc/flink entropy
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/6302.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #6302
----
commit 96def47c57ae58f9717ab35d41e1a4bf6d152851
Author: Indrajit Roychoudhury <iroychoudhury@...>
Date: 2018-06-26T06:36:49Z
[FLINK-9061][checkpointing] add entropy to s3 path for better scalability
commit 15c06fa70777e763bac4bde67a48d1d40b5d15ba
Author: Indrajit Roychoudhury <iroychoudhury@...>
Date: 2018-07-11T03:16:45Z
Add tests, address review comments
----
> 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)