Samrat002 commented on code in PR #27841:
URL: https://github.com/apache/flink/pull/27841#discussion_r3031360620
##########
docs/content/docs/deployment/filesystems/s3.md:
##########
@@ -64,94 +64,267 @@ env.configure(config);
Note that these examples are *not* exhaustive and you can use S3 in other
places as well, including your [high availability setup]({{< ref
"docs/deployment/ha/overview" >}}) or the [EmbeddedRocksDBStateBackend]({{< ref
"docs/ops/state/state_backends" >}}#the-rocksdbstatebackend); everywhere that
Flink expects a FileSystem URI (unless otherwise stated).
-For most use cases, you may use one of our `flink-s3-fs-hadoop` and
`flink-s3-fs-presto` S3 filesystem plugins which are self-contained and easy to
set up.
-For some cases, however, e.g., for using S3 as YARN's resource storage dir, it
may be necessary to set up a specific Hadoop S3 filesystem implementation.
+## S3 FileSystem Implementations
-### Hadoop/Presto S3 File Systems plugins
+Flink provides three independent S3 filesystem implementations, each with
different trade-offs:
-{{< hint info >}}
-You don't have to configure this manually if you are running [Flink on
EMR](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html).
-{{< /hint >}}
-
-Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and
`flink-s3-fs-hadoop`.
-Both implementations are self-contained with no dependency footprint, so there
is no need to add Hadoop to the classpath to use them.
+- **Native S3 FileSystem** (`flink-s3-fs-native`): Built directly on AWS SDK
v2 with async I/O and parallel transfers, this implementation supports both
checkpointing and the FileSystem sink.
[Benchmarks](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=406620396)
show ~2x higher checkpoint throughput (~200 MB/s vs ~90 MB/s) compared to the
Presto implementation at state sizes up to 15 GB. **Experimental** in Flink
2.3; the API and behavior may change in future releases.
+- **Presto S3 FileSystem** (`flink-s3-fs-presto`): Based on Presto project
code, recommended for checkpointing.
+- **Hadoop S3 FileSystem** (`flink-s3-fs-hadoop`): Based on Hadoop project
code, has FileSystem sink support.
- - `flink-s3-fs-presto`, registered under the scheme *s3://* and *s3p://*, is
based on code from the [Presto project](https://prestodb.io/).
- You can configure it using [the same configuration keys as the Presto file
system](https://prestodb.io/docs/0.272/connector/hive.html#amazon-s3-configuration),
by adding the configurations to your [Flink configuration file]({{< ref
"docs/deployment/config#flink-configuration-file" >}}). The Presto S3
implementation is the recommended file system for checkpointing to S3.
+All three are self-contained with no dependency footprint, so there is no need
to add Hadoop to the classpath to use them.
- - `flink-s3-fs-hadoop`, registered under *s3://* and *s3a://*, based on code
from the [Hadoop Project](https://hadoop.apache.org/).
- The file system can be [configured using Hadoop's s3a configuration
keys](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A)
by adding the configurations to your [Flink configuration file]({{< ref
"docs/deployment/config#flink-configuration-file" >}}).
-
- For example, Hadoop has a `fs.s3a.connection.maximum` configuration key.
If you want to change it, you need to put `s3.connection.maximum: xyz` to the
[Flink configuration file]({{< ref
"docs/deployment/config#flink-configuration-file" >}}). Flink will internally
translate this back to `fs.s3a.connection.maximum`. There is no need to pass
configuration parameters using Hadoop's XML configuration files.
-
- It is the only S3 file system with support for the [FileSystem]({{< ref
"docs/connectors/datastream/filesystem" >}}).
-
+## Common Configuration
-Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
-wrappers for URIs with the *s3://* scheme, `flink-s3-fs-hadoop` also registers
-for *s3a://* and `flink-s3-fs-presto` also registers for *s3p://*, so you can
-use this to use both at the same time.
-For example, the job uses the [FileSystem]({{< ref
"docs/connectors/datastream/filesystem" >}}) which only supports Hadoop, but
uses Presto for checkpointing.
-In this case, you should explicitly use *s3a://* as a scheme for the sink
(Hadoop) and *s3p://* for checkpointing (Presto).
+### Configure Access Credentials
-To use `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR
file from the `opt` directory to the `plugins` directory of your Flink
distribution before starting Flink, e.g.
-
-```bash
-mkdir ./plugins/s3-fs-presto
-cp ./opt/flink-s3-fs-presto-{{< version >}}.jar ./plugins/s3-fs-presto/
-```
+After setting up the S3 FileSystem implementation, you need to make sure that
Flink is allowed to access your S3 buckets.
-#### Configure Access Credentials
-
-After setting up the S3 FileSystem wrapper, you need to make sure that Flink
is allowed to access your S3 buckets.
-
-##### Identity and Access Management (IAM) (Recommended)
+#### Identity and Access Management (IAM) (Recommended)
The recommended way of setting up credentials on AWS is via [Identity and
Access Management
(IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You
can use IAM features to securely give Flink instances the credentials that they
need to access S3 buckets. Details about how to do this are beyond the scope of
this documentation. Please refer to the AWS user guide. What you are looking
for are [IAM
Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html).
If you set this up correctly, you can manage access to S3 within AWS and don't
need to distribute any access keys to Flink.
-##### Access Keys (Discouraged)
+#### Access Keys (Discouraged)
Access to S3 can be granted via your **access and secret key pair**. Please
note that this is discouraged since the [introduction of IAM
roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-You need to configure both `s3.access-key` and `s3.secret-key` in Flink's
[Flink configuration file]({{< ref
"docs/deployment/config#flink-configuration-file" >}}):
+You need to configure both `s3.access-key` and `s3.secret-key` in Flink's
[configuration file]({{< ref "docs/deployment/config#flink-configuration-file"
>}}):
```yaml
s3.access-key: your-access-key
s3.secret-key: your-secret-key
```
-You can limit this configuration to JobManagers by using [Flink configuration
file]({{< ref "docs/deployment/security/security-delegation-token" >}}).
+You can limit this configuration to JobManagers by using [delegation
tokens]({{< ref "docs/deployment/security/security-delegation-token" >}}):
```yaml
-# flink-s3-fs-hadoop
-fs.s3a.aws.credentials.provider:
org.apache.flink.fs.s3.common.token.DynamicTemporaryAWSCredentialsProvider
-# flink-s3-fs-presto
+# For Native S3 or Hadoop implementations
+fs.s3.aws.credentials.provider:
org.apache.flink.fs.s3.common.token.DynamicTemporaryAWSCredentialsProvider
Review Comment:
updated
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]