Skip to content

Commit

Permalink
[FLINK-12378][docs] Consolidate FileSystem Documentation
Browse files Browse the repository at this point in the history
  • Loading branch information
sjwiesman authored and StephanEwen committed May 10, 2019
1 parent 446e711 commit b50896a
Show file tree
Hide file tree
Showing 12 changed files with 544 additions and 255 deletions.
74 changes: 2 additions & 72 deletions docs/ops/deployment/aws.md
Original file line number Diff line number Diff line change
Expand Up @@ -64,81 +64,11 @@ HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 1 examples/

{% top %}

## S3: Simple Storage Service

[Amazon Simple Storage Service](https://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html) or even as a YARN object storage.

You can use S3 objects like regular files by specifying paths in the following format:

{% highlight plain %}
s3:https://<your-bucket>/<endpoint>
{% endhighlight %}

The endpoint can either be a single file or a directory, for example:

{% highlight java %}
// Read from S3 bucket
env.readTextFile("s3:https://<bucket>/<endpoint>");

// Write to S3 bucket
stream.writeAsText("s3:https://<bucket>/<endpoint>");

// Use S3 as FsStatebackend
env.setStateBackend(new FsStateBackend("s3:https://<your-bucket>/<endpoint>"));
{% endhighlight %}

Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI.

For most use cases, you may use one of our shaded `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3
filesystem wrappers which are fairly 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. Both ways are described below.

### Shaded Hadoop/Presto S3 file systems (recommended)

{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}

To use either `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the
`opt` directory to the `lib` directory of your Flink distribution before starting Flink, e.g.

{% highlight bash %}
cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./lib/
{% endhighlight %}

Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem
wrappers for URIs with the `s3:https://` scheme, `flink-s3-fs-hadoop` also registers
for `s3a:https://` and `flink-s3-fs-presto` also registers for `s3p:https://`, so you can
use this to use both at the same time.

#### 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)

The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](https://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need in order 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](https://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 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-conf.yaml`:

{% highlight yaml %}
s3.access-key: your-access-key
s3.secret-key: your-secret-key
{% endhighlight %}

{% top %}

### Hadoop-provided S3 file systems - manual setup
### Hadoop-provided S3 file systems

{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](#emr-elastic-mapreduce). %}

This setup is a bit more complex and we recommend using our shaded Hadoop/Presto file systems
instead (see above) unless required otherwise, e.g. for using S3 as YARN's resource storage dir
Apache Flink provides native [S3 FileSystem's](../filesystems/s3.html) out of the box and we recomend using them unless required otherwise, e.g. for using S3 as YARN's resource storage dir
via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`.

#### Set S3 FileSystem
Expand Down
177 changes: 0 additions & 177 deletions docs/ops/filesystems.md

This file was deleted.

68 changes: 68 additions & 0 deletions docs/ops/filesystems/common.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
---
title: "Common Configurations"
nav-parent_id: filesystems
nav-pos: 0
---
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
https://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->

Apache Flink provides a number of common configuration settings that work across all file system implementations.

* This will be replaced by the TOC
{:toc}

## Default File System

If paths to files do not explicitly specify a file system scheme (and authority), a default scheme (and authority) will be used.

{% highlight yaml %}
fs.default-scheme: <default-fs>
{% endhighlight %}

For example, if the default file system configured as `fs.default-scheme: hdfs:https://localhost:9000/`, then a file path of
`/user/hugo/in.txt` is interpreted as `hdfs:https://localhost:9000/user/hugo/in.txt`.

## Connection limiting

You can limit the total number of connections that a file system can concurrently open. This is useful when the file system cannot handle a large number
of concurrent reads / writes or open connections at the same time.

For example, very small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint.

To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by
its scheme.

{% highlight yaml %}
fs.<scheme>.limit.total: (number, 0/-1 mean no limit)
fs.<scheme>.limit.input: (number, 0/-1 mean no limit)
fs.<scheme>.limit.output: (number, 0/-1 mean no limit)
fs.<scheme>.limit.timeout: (milliseconds, 0 means infinite)
fs.<scheme>.limit.stream-timeout: (milliseconds, 0 means infinite)
{% endhighlight %}

You can limit the number if input/output connections (streams) separately (`fs.<scheme>.limit.input` and `fs.<scheme>.limit.output`), as well as impose a limit on
the total number of concurrent streams (`fs.<scheme>.limit.total`). If the file system tries to open more streams, the operation will block until some streams are closed.
If the opening of the stream takes longer than `fs.<scheme>.limit.timeout`, the stream opening will fail.

To prevent inactive streams from taking up the complete pool (preventing new connections to be opened), you can add an inactivity timeout for streams:
`fs.<scheme>.limit.stream-timeout`. If a stream does not read/write any bytes for at least that amount of time, it is forcibly closed.

These limits are enforced per TaskManager, so each TaskManager in a Flink application or cluster will open up to that number of connections.
In addition, the limits are also only enforced per FileSystem instance. Because File Systems are created per scheme and authority, different
authorities will have their own connection pool. For example `hdfs:https://myhdfs:50010/` and `hdfs:https://anotherhdfs:4399/` will have separate pools.
Loading

0 comments on commit b50896a

Please sign in to comment.