[
https://issues.apache.org/jira/browse/FLINK-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16622275#comment-16622275
]
ASF GitHub Bot commented on FLINK-9061:
---------------------------------------
StephanEwen opened a new pull request #6726: [FLINK-9061] [s3] Add entropy
injection to S3 file systems for checkpoints
URL: https://github.com/apache/flink/pull/6726
## What is the purpose of the change
This pull request adds hooks to optionally inject entropy to the checkpoint
path based upon a user defined pattern in the configuration for better S3
scalability.
This is a revised version of #6302 by @indrc (based on Flink 1.4) that ports
the changes to the new FileStateStorage code introduced in Flink 1.5 and to the
new S3 common abstraction recently added to the master branch.
The difference to the previous approach is that the entropy injection sits
only in the S3 file system code and the checkpointing code only passes a write
option that says that entropy is desired here.
The advantage of that is that the checkpointing code stays simple.
Furthermore, entropy injection is only configurable for S3, which is the only
FS that can support this style of path rewriting properly, because it is
actually an object store and can handle random injection in the paths without
causing "parent does not exist" directory problems.
## Brief change log
- Extends the FileSystem class with a `write(Path, WriteOptions)` method.
This method should subsume all special methods with extra options that are
relevant to a subset of file systems. The `WriteOptions` object makes this
extensible without breaking the API.
- Adds an implementation of entropy injection for the base S3 file system
from which the Presto-based and Hadoop-based S3 file systems inherit. When
entropy injection is configured and requested, the entropy key is replaced in
the path, otherwise filtered out.
- The only change in checkpointing is that the `FsCheckpointStreamFactory`
passes the specific WriteOption to "inject entropy here if configured".
## Verifying this change
- Added unit test for entropy injection under
`org.apache.flink.runtime.state.filesystem`
- Added unit test for checkpointing under `org.apache.flink.fs.s3.common`
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): **no**
- 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: **no**
- The S3 file system connector: **yes**
## Documentation
- Does this pull request introduce a new feature? **yes**
- If yes, how is the feature documented? **docs**
----------------------------------------------------------------
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:
[email protected]
> 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)