http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/state/large_state_tuning.md
----------------------------------------------------------------------
diff --git a/docs/ops/state/large_state_tuning.md 
b/docs/ops/state/large_state_tuning.md
new file mode 100644
index 0000000..aa0b0d8
--- /dev/null
+++ b/docs/ops/state/large_state_tuning.md
@@ -0,0 +1,237 @@
+---
+title: "Debugging and Tuning Checkpoints and Large State"
+nav-parent_id: ops_state
+nav-pos: 12
+---
+<!--
+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
+
+  http://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.
+-->
+
+This page gives a guide how to configure and tune applications that use large 
state.
+
+* ToC
+{:toc}
+
+## Overview
+
+For Flink applications to run reliably at large scale, two conditions must be 
fulfilled:
+
+  - The application needs to be able to take checkpoints reliably
+
+  - The resources need to be sufficient catch up with the input data streams 
after a failure
+
+The first sections discuss how to get well performing checkpoints at scale.
+The last section explains some best practices concerning planning how many 
resources to use.
+
+
+## Monitoring State and Checkpoints
+
+The easiest way to monitor checkpoint behavior is via the UI's checkpoint 
section. The documentation
+for [checkpoint monitoring](../../monitoring/checkpoint_monitoring.html) shows 
how to access the available checkpoint
+metrics.
+
+The two numbers that are of particular interest when scaling up checkpoints 
are:
+
+  - The time until operators start their checkpoint: This time is currently 
not exposed directly, but corresponds
+    to:
+    
+    `checkpoint_start_delay = end_to_end_duration - synchronous_duration - 
asynchronous_duration`
+
+    When the time to trigger the checkpoint is constantly very high, it means 
that the *checkpoint barriers* need a long
+    time to travel from the source to the operators. That typically indicates 
that the system is operating under a
+    constant backpressure.
+
+  - The amount of data buffered during alignments. For exactly-once semantics, 
Flink *aligns* the streams at
+    operators that receive multiple input streams, buffering some data for 
that alignment.
+    The buffered data volume is ideally low - higher amounts means that 
checkpoint barriers are reveived at
+    very different times from the different input streams.
+
+Note that when the here indicated numbers can be occasionally high in the 
presence of transient backpressure, data skew,
+or network issues. However, if the numbers are constantly very high, it means 
that Flink puts many resources into checkpointing.
+
+
+## Tuning Checkpointing
+
+Checkpoints are triggered at regular intervals that applications can 
configure. When a checkpoint takes longer
+to complete than the checkpoint interval, the next checkpoint is not triggered 
before the in-progress checkpoint
+completes. By default the next checkpoint will then be triggered immediately 
once the ongoing checkpoint completes.
+
+When checkpoints end up frequently taking longer than the base interval (for 
example because state
+grew larger than planned, or the storage where checkpoints are stored is 
temporarily slow),
+the system is constantly taking checkpoints (new ones are started immediately 
once ongoing once finish).
+That can mean that too many resources are constantly tied up in checkpointing 
and that the operators make too
+little progress. This behavior has less impact on streaming applications that 
use asynchronously checkpointed state,
+but may still have an impact on overall application performance.
+
+To prevent such a situation, applications can define a *minimum duration 
between checkpoints*:
+
+`StreamExecutionEnvironment.getCheckpointConfig().setMinPauseBetweenCheckpoints(milliseconds)`
+
+This duration is the minimum time interval that must pass between the end of 
the latest checkpoint and the beginning
+of the next. The figure below illustrates how this impacts checkpointing.
+
+<img src="../../fig/checkpoint_tuning.svg" class="center" width="80%" 
alt="Illustration how the minimum-time-between-checkpoints parameter affects 
checkpointing behavior."/>
+
+*Note:* Applications can be configured (via the `CheckpointConfig`) to allow 
multiple checkpoints to be in progress at
+the same time. For applications with large state in Flink, this often ties up 
too many resources into the checkpointing.
+When a savepoint is manually triggered, it may be in process concurrently with 
an ongoing checkpoint.
+
+
+## Tuning Network Buffers
+
+Before Flink 1.3, an increased number of network buffers also caused increased 
checkpointing times since
+keeping more in-flight data meant that checkpoint barriers got delayed. Since 
Flink 1.3, the
+number of network buffers used per outgoing/incoming channel is limited and 
thus network buffers
+may be configured without affecting checkpoint times
+(see [network buffer 
configuration](../config.html#configuring-the-network-buffers)).
+
+## Make state checkpointing Asynchronous where possible
+
+When state is *asynchronously* snapshotted, the checkpoints scale better than 
when the state is *synchronously* snapshotted.
+Especially in more complex streaming applications with multiple joins, 
Co-functions, or windows, this may have a profound
+impact.
+
+To get state to be snapshotted asynchronously, applications have to do two 
things:
+
+  1. Use state that is [managed by Flink](../../dev/stream/state/state.html): 
Managed state means that Flink provides the data
+     structure in which the state is stored. Currently, this is true for 
*keyed state*, which is abstracted behind the
+     interfaces like `ValueState`, `ListState`, `ReducingState`, ...
+
+  2. Use a state backend that supports asynchronous snapshots. In Flink 1.2, 
only the RocksDB state backend uses
+     fully asynchronous snapshots.
+
+The above two points imply that (in Flink 1.2) large state should generally be 
kept as keyed state, not as operator state.
+This is subject to change with the planned introduction of *managed operator 
state*.
+
+
+## Tuning RocksDB
+
+The state storage workhorse of many large scale Flink streaming applications 
is the *RocksDB State Backend*.
+The backend scales well beyond main memory and reliably stores large [keyed 
state](../../dev/stream/state/state.html).
+
+Unfortunately, RocksDB's performance can vary with configuration, and there is 
little documentation on how to tune
+RocksDB properly. For example, the default configuration is tailored towards 
SSDs and performs suboptimal
+on spinning disks.
+
+**Incremental Checkpoints**
+
+Incremental checkpoints can dramatically reduce the checkpointing time in 
comparison to full checkpoints, at the cost of a (potentially) longer
+recovery time. The core idea is that incremental checkpoints only record all 
changes to the previous completed checkpoint, instead of
+producing a full, self-contained backup of the state backend. Like this, 
incremental checkpoints build upon previous checkpoints. Flink leverages
+RocksDB's internal backup mechanism in a way that is self-consolidating over 
time. As a result, the incremental checkpoint history in Flink
+does not grow indefinitely, and old checkpoints are eventually subsumed and 
pruned automatically. `
+
+While we strongly encourage the use of incremental checkpoints for large 
state, please note that this is a new feature and currently not enabled 
+by default. To enable this feature, users can instantiate a 
`RocksDBStateBackend` with the corresponding boolean flag in the constructor 
set to `true`, e.g.:
+
+{% highlight java %}
+    RocksDBStateBackend backend =
+        new RocksDBStateBackend(filebackend, true);
+{% endhighlight %}
+
+**Passing Options to RocksDB**
+
+{% highlight java %}
+RocksDBStateBackend.setOptions(new MyOptions());
+
+public class MyOptions implements OptionsFactory {
+
+    @Override
+    public DBOptions createDBOptions() {
+        return new DBOptions()
+            .setIncreaseParallelism(4)
+            .setUseFsync(false)
+            .setDisableDataSync(true);
+    }
+
+    @Override
+    public ColumnFamilyOptions createColumnOptions() {
+
+        return new ColumnFamilyOptions()
+            .setTableFormatConfig(
+                new BlockBasedTableConfig()
+                    .setBlockCacheSize(256 * 1024 * 1024)  // 256 MB
+                    .setBlockSize(128 * 1024));            // 128 KB
+    }
+}
+{% endhighlight %}
+
+**Predefined Options**
+
+Flink provides some predefined collections of option for RocksDB for different 
settings, which can be set for example via
+`RocksDBStateBacked.setPredefinedOptions(PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM)`.
+
+We expect to accumulate more such profiles over time. Feel free to contribute 
such predefined option profiles when you
+found a set of options that work well and seem representative for certain 
workloads.
+
+**Important:** RocksDB is a native library, whose allocated memory not from 
the JVM, but directly from the process'
+native memory. Any memory you assign to RocksDB will have to be accounted for, 
typically by decreasing the JVM heap size
+of the TaskManagers by the same amount. Not doing that may result in 
YARN/Mesos/etc terminating the JVM processes for
+allocating more memory than configures.
+
+
+## Capacity Planning
+
+This section discusses how to decide how many resources should be used for a 
Flink job to run reliably.
+The basic rules of thumb for capacity planning are:
+
+  - Normal operation should have enough capacity to not operate under constant 
*back pressure*.
+    See [back pressure monitoring](../../monitoring/back_pressure.html) for 
details on how to check whether the application runs under back pressure.
+
+  - Provision some extra resources on top of the resources needed to run the 
program back-pressure-free during failure-free time.
+    These resources are needed to "catch up" with the input data that 
accumulated during the time the application
+    was recovering.
+    How much that should be depends on how long recovery operations usually 
take (which depends on the size of the state
+    that needs to be loaded into the new TaskManagers on a failover) and how 
fast the scenario requires failures to recover.
+
+    *Important*: The base line should to be established with checkpointing 
activated, because checkpointing ties up
+    some amount of resources (such as network bandwidth).
+
+  - Temporary back pressure is usually okay, and an essential part of 
execution flow control during load spikes,
+    during catch-up phases, or when external systems (that are written to in a 
sink) exhibit temporary slowdown.
+
+  - Certain operations (like large windows) result in a spiky load for their 
downstream operators: 
+    In the case of windows, the downstream operators may have little to do 
while the window is being built,
+    and have a load to do when the windows are emitted.
+    The planning for the downstream parallelism needs to take into account how 
much the windows emit and how
+    fast such a spike needs to be processed.
+
+**Important:** In order to allow for adding resources later, make sure to set 
the *maximum parallelism* of the
+data stream program to a reasonable number. The maximum parallelism defines 
how high you can set the programs
+parallelism when re-scaling the program (via a savepoint).
+
+Flink's internal bookkeeping tracks parallel state in the granularity of 
max-parallelism-many *key groups*.
+Flink's design strives to make it efficient to have a very high value for the 
maximum parallelism, even if
+executing the program with a low parallelism.
+
+## Compression
+
+Flink offers optional compression (default: off) for all checkpoints and 
savepoints. Currently, compression always uses 
+the [snappy compression algorithm (version 
1.1.4)](https://github.com/xerial/snappy-java) but we are planning to support
+custom compression algorithms in the future. Compression works on the 
granularity of key-groups in keyed state, i.e.
+each key-group can be decompressed individually, which is important for 
rescaling. 
+
+Compression can be activated through the `ExecutionConfig`:
+
+{% highlight java %}
+               ExecutionConfig executionConfig = new ExecutionConfig();
+               executionConfig.setUseSnapshotCompression(true);
+{% endhighlight %}
+
+**Notice:** The compression option has no impact on incremental snapshots, 
because they are using RocksDB's internal
+format which is always using snappy compression out of the box.

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/state/savepoints.md
----------------------------------------------------------------------
diff --git a/docs/ops/state/savepoints.md b/docs/ops/state/savepoints.md
new file mode 100644
index 0000000..1d82d2b
--- /dev/null
+++ b/docs/ops/state/savepoints.md
@@ -0,0 +1,198 @@
+---
+title: "Savepoints"
+nav-parent_id: ops_state
+nav-pos: 8
+---
+<!--
+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
+
+  http://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.
+-->
+
+* toc
+{:toc}
+
+## Overview
+
+Savepoints are externally stored self-contained checkpoints that you can use 
to stop-and-resume or update your Flink programs. They use Flink's 
[checkpointing mechanism]({{ site.baseurl 
}}/internals/stream_checkpointing.html) to create a (non-incremental) snapshot 
of the state of your streaming program and write the checkpoint data and meta 
data out to an external file system.
+
+This page covers all steps involved in triggering, restoring, and disposing 
savepoints.
+For more details on how Flink handles state and failures in general, check out 
the [State in Streaming Programs]({{ site.baseurl 
}}/dev/stream/state/index.html) page.
+
+<div class="alert alert-warning">
+<strong>Attention:</strong> In order to allow upgrades between programs and 
Flink versions, it is important to check out the following section about <a 
href="#assigning-operator-ids">assigning IDs to your operators</a>.
+</div>
+
+## Assigning Operator IDs
+
+It is **highly recommended** that you adjust your programs as described in 
this section in order to be able to upgrade your programs in the future. The 
main required change is to manually specify operator IDs via the 
**`uid(String)`** method. These IDs are used to scope the state of each 
operator.
+
+{% highlight java %}
+DataStream<String> stream = env.
+  // Stateful source (e.g. Kafka) with ID
+  .addSource(new StatefulSource())
+  .uid("source-id") // ID for the source operator
+  .shuffle()
+  // Stateful mapper with ID
+  .map(new StatefulMapper())
+  .uid("mapper-id") // ID for the mapper
+  // Stateless printing sink
+  .print(); // Auto-generated ID
+{% endhighlight %}
+
+If you don't specify the IDs manually they will be generated automatically. 
You can automatically restore from the savepoint as long as these IDs do not 
change. The generated IDs depend on the structure of your program and are 
sensitive to program changes. Therefore, it is highly recommended to assign 
these IDs manually.
+
+### Savepoint State
+
+You can think of a savepoint as holding a map of `Operator ID -> State` for 
each stateful operator:
+
+```
+Operator ID | State
+------------+------------------------
+source-id   | State of StatefulSource
+mapper-id   | State of StatefulMapper
+```
+
+In the above example, the print sink is stateless and hence not part of the 
savepoint state. By default, we try to map each entry of the savepoint back to 
the new program.
+
+## Operations
+
+You can use the [command line client]({{ site.baseurl 
}}/ops/cli.html#savepoints) to *trigger savepoints*, *cancel a job with a 
savepoint*, *resume from savepoints*, and *dispose savepoints*.
+
+With Flink >= 1.2.0 it is also possible to *resume from savepoints* using the 
webui.
+
+### Triggering Savepoints
+
+When triggering a savepoint, a new savepoint directory beneath the target 
directory is created. In there, the data as well as the meta data will be 
stored. For example with a `FsStateBackend` or `RocksDBStateBackend`:
+
+```sh
+# Savepoint target directory
+/savepoints/
+
+# Savepoint directory
+/savepoints/savepoint-:shortjobid-:savepointid/
+
+# Savepoint file contains the checkpoint meta data
+/savepoints/savepoint-:shortjobid-:savepointid/_metadata
+
+# Savepoint state
+/savepoints/savepoint-:shortjobid-:savepointid/...
+```
+
+<div class="alert alert-info">
+  <strong>Note:</strong>
+Although it looks as if the savepoints may be moved, it is currently not 
possible due to absolute paths in the <code>_metadata</code> file.
+Please follow <a 
href="https://issues.apache.org/jira/browse/FLINK-5778";>FLINK-5778</a> for 
progress on lifting this restriction.
+</div>
+
+Note that if you use the `MemoryStateBackend`, metadata *and* savepoint state 
will be stored in the `_metadata` file. Since it is self-contained, you may 
move the file and restore from any location.
+
+#### Trigger a Savepoint
+
+```sh
+$ bin/flink savepoint :jobId [:targetDirectory]
+```
+
+This will trigger a savepoint for the job with ID `:jobid`. Furthermore, you 
can specify a target file system directory to store the savepoint in. The 
directory needs to be accessible by the JobManager.
+
+If you don't specify a target directory, you need to have [configured a 
default directory](#configuration). Otherwise, triggering the savepoint will 
fail.
+
+#### Cancel Job with Savepoint
+
+```sh
+$ bin/flink cancel -s [:targetDirectory] :jobId
+```
+
+This will atomically trigger a savepoint for the job with ID `:jobid` and 
cancel the job. Furthermore, you can specify a target file system directory to 
store the savepoint in.  The directory needs to be accessible by the JobManager.
+
+If you don't specify a target directory, you need to have [configured a 
default directory](#configuration). Otherwise, cancelling the job with a 
savepoint will fail.
+
+### Resuming from Savepoints
+
+```sh
+$ bin/flink run -s :savepointPath [:runArgs]
+```
+
+This submits a job and specifies a savepoint to resume from. You may give a 
path to either the savepoint's directory or the `_metadata` file.
+
+#### Allowing Non-Restored State
+
+By default the resume operation will try to map all state of the savepoint 
back to the program you are restoring with. If you dropped an operator, you can 
allow to skip state that cannot be mapped to the new program via 
`--allowNonRestoredState` (short: `-n`) option:
+
+```sh
+$ bin/flink run -s :savepointPath -n [:runArgs]
+```
+
+### Disposing Savepoints
+
+```sh
+$ bin/flink savepoint -d :savepointPath
+```
+
+This disposes the savepoint stored in `:savepointPath`.
+
+Note that it is possible to also manually delete a savepoint via regular file 
system operations without affecting other savepoints or checkpoints (recall 
that each savepoint is self-contained). Up to Flink 1.2, this was a more 
tedious task which was performed with the savepoint command above.
+
+### Configuration
+
+You can configure a default savepoint target directory via the 
`state.savepoints.dir` key. When triggering savepoints, this directory will be 
used to store the savepoint. You can overwrite the default by specifying a 
custom target directory with the trigger commands (see the [`:targetDirectory` 
argument](#trigger-a-savepoint)).
+
+```sh
+# Default savepoint target directory
+state.savepoints.dir: hdfs:///flink/savepoints
+```
+
+If you neither configure a default nor specify a custom target directory, 
triggering the savepoint will fail.
+
+## F.A.Q
+
+### Should I assign IDs to all operators in my job?
+
+As a rule of thumb, yes. Strictly speaking, it is sufficient to only assign 
IDs via the `uid` method to the stateful operators in your job. The savepoint 
only contains state for these operators and stateless operator are not part of 
the savepoint.
+
+In practice, it is recommended to assign it to all operators, because some of 
Flink's built-in operators like the Window operator are also stateful and it is 
not obvious which built-in operators are actually stateful and which are not. 
If you are absolutely certain that an operator is stateless, you can skip the 
`uid` method.
+
+### What happens if I add a new operator that requires state to my job?
+
+When you add a new operator to your job it will be initialized without any 
state. Savepoints contain the state of each stateful operator. Stateless 
operators are simply not part of the savepoint. The new operator behaves 
similar to a stateless operator.
+
+### What happens if I delete an operator that has state from my job?
+
+By default, a savepoint restore will try to match all state back to the 
restored job. If you restore from a savepoint that contains state for an 
operator that has been deleted, this will therefore fail. 
+
+You can allow non restored state by setting the `--allowNonRestoredState` 
(short: `-n`) with the run command:
+
+```sh
+$ bin/flink run -s :savepointPath -n [:runArgs]
+```
+
+### What happens if I reorder stateful operators in my job?
+
+If you assigned IDs to these operators, they will be restored as usual.
+
+If you did not assign IDs, the auto generated IDs of the stateful operators 
will most likely change after the reordering. This would result in you not 
being able to restore from a previous savepoint.
+
+### What happens if I add or delete or reorder operators that have no state in 
my job?
+
+If you assigned IDs to your stateful operators, the stateless operators will 
not influence the savepoint restore.
+
+If you did not assign IDs, the auto generated IDs of the stateful operators 
will most likely change after the reordering. This would result in you not 
being able to restore from a previous savepoint.
+
+### What happens when I change the parallelism of my program when restoring?
+
+If the savepoint was triggered with Flink >= 1.2.0 and using no deprecated 
state API like `Checkpointed`, you can simply restore the program from a 
savepoint and specify a new parallelism.
+
+If you are resuming from a savepoint triggered with Flink < 1.2.0 or using now 
deprecated APIs you first have to migrate your job and savepoint to Flink >= 
1.2.0 before being able to change the parallelism. See the [upgrading jobs and 
Flink versions guide]({{ site.baseurl }}/ops/upgrading.html).

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/state/state_backends.md
----------------------------------------------------------------------
diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md
new file mode 100644
index 0000000..b53bcef
--- /dev/null
+++ b/docs/ops/state/state_backends.md
@@ -0,0 +1,169 @@
+---
+title: "State Backends"
+nav-parent_id: ops_state
+nav-pos: 11
+---
+<!--
+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
+
+  http://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.
+-->
+
+Programs written in the [Data Stream API]({{ site.baseurl 
}}/dev/datastream_api.html) often hold state in various forms:
+
+- Windows gather elements or aggregates until they are triggered
+- Transformation functions may use the key/value state interface to store 
values
+- Transformation functions may implement the `Checkpointed` interface to make 
their local variables fault tolerant
+
+See also [state section]({{ site.baseurl }}/dev/stream/state/index.html) in 
the streaming API guide.
+
+When checkpointing is activated, such state is persisted upon checkpoints to 
guard against data loss and recover consistently.
+How the state is represented internally, and how and where it is persisted 
upon checkpoints depends on the
+chosen **State Backend**.
+
+* ToC
+{:toc}
+
+## Available State Backends
+
+Out of the box, Flink bundles these state backends:
+
+ - *MemoryStateBackend*
+ - *FsStateBackend*
+ - *RocksDBStateBackend*
+
+If nothing else is configured, the system will use the MemoryStateBackend.
+
+
+### The MemoryStateBackend
+
+The *MemoryStateBackend* holds data internally as objects on the Java heap. 
Key/value state and window operators hold hash tables
+that store the values, triggers, etc.
+
+Upon checkpoints, this state backend will snapshot the state and send it as 
part of the checkpoint acknowledgement messages to the
+JobManager (master), which stores it on its heap as well.
+
+The MemoryStateBackend can be configured to use asynchronous snapshots. While 
we strongly encourage the use of asynchronous snapshots to avoid blocking 
pipelines, please note that this is a new feature and currently not enabled 
+by default. To enable this feature, users can instantiate a 
`MemoryStateBackend` with the corresponding boolean flag in the constructor set 
to `true`, e.g.:
+
+{% highlight java %}
+    new MemoryStateBackend(MAX_MEM_STATE_SIZE, true);
+{% endhighlight %}
+
+Limitations of the MemoryStateBackend:
+
+  - The size of each individual state is by default limited to 5 MB. This 
value can be increased in the constructor of the MemoryStateBackend.
+  - Irrespective of the configured maximal state size, the state cannot be 
larger than the akka frame size (see [Configuration]({{ site.baseurl 
}}/ops/config.html)).
+  - The aggregate state must fit into the JobManager memory.
+
+The MemoryStateBackend is encouraged for:
+
+  - Local development and debugging
+  - Jobs that do hold little state, such as jobs that consist only of 
record-at-a-time functions (Map, FlatMap, Filter, ...). The Kafka Consumer 
requires very little state.
+
+
+### The FsStateBackend
+
+The *FsStateBackend* is configured with a file system URL (type, address, 
path), such as "hdfs://namenode:40010/flink/checkpoints" or 
"file:///data/flink/checkpoints".
+
+The FsStateBackend holds in-flight data in the TaskManager's memory. Upon 
checkpointing, it writes state snapshots into files in the configured file 
system and directory. Minimal metadata is stored in the JobManager's memory 
(or, in high-availability mode, in the metadata checkpoint).
+
+The FsStateBackend can be configured to use asynchronous snapshots. While we 
strongly encourage the use of asynchronous snapshots to avoid blocking 
pipelines, please note that this is a new feature and currently not enabled 
+by default. To enable this feature, users can instantiate a `FsStateBackend` 
with the corresponding boolean flag in the constructor set to `true`, e.g.:
+
+{% highlight java %}
+    new FsStateBackend(path, true);
+{% endhighlight %}
+
+The FsStateBackend is encouraged for:
+
+  - Jobs with large state, long windows, large key/value states.
+  - All high-availability setups.
+
+### The RocksDBStateBackend
+
+The *RocksDBStateBackend* is configured with a file system URL (type, address, 
path), such as "hdfs://namenode:40010/flink/checkpoints" or 
"file:///data/flink/checkpoints".
+
+The RocksDBStateBackend holds in-flight data in a 
[RocksDB](http://rocksdb.org) data base
+that is (per default) stored in the TaskManager data directories. Upon 
checkpointing, the whole
+RocksDB data base will be checkpointed into the configured file system and 
directory. Minimal
+metadata is stored in the JobManager's memory (or, in high-availability mode, 
in the metadata checkpoint).
+
+The RocksDBStateBackend always performs asynchronous snapshots.
+
+Limitations of the RocksDBStateBackend:
+
+  - As RocksDB's JNI bridge API is based on byte[], the maximum supported size 
per key and per value is 2^31 bytes each. 
+  IMPORTANT: states that use merge operations in RocksDB (e.g. ListState) can 
silently accumulate value sizes > 2^31 bytes and will then fail on their next 
retrieval. This is currently a limitation of RocksDB JNI.
+
+The RocksDBStateBackend is encouraged for:
+
+  - Jobs with very large state, long windows, large key/value states.
+  - All high-availability setups.
+
+Note that the amount of state that you can keep is only limited by the amount 
of disc space available.
+This allows keeping very large state, compared to the FsStateBackend that 
keeps state in memory.
+This also means, however, that the maximum throughput that can be achieved 
will be lower with
+this state backend.
+
+RocksDBStateBackend is currently the only backend that offers incremental 
checkpoints (see [here](large_state_tuning.html)). 
+
+## Configuring a State Backend
+
+The default state backend, if you specify nothing, is the jobmanager. If you 
wish to establish a different default for all jobs on your cluster, you can do 
so by defining a new default state backend in **flink-conf.yaml**. The default 
state backend can be overridden on a per-job basis, as shown below.
+
+### Setting the Per-job State Backend
+
+The per-job state backend is set on the `StreamExecutionEnvironment` of the 
job, as shown in the example below:
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+env.setStateBackend(new 
FsStateBackend("hdfs://namenode:40010/flink/checkpoints"));
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = StreamExecutionEnvironment.getExecutionEnvironment()
+env.setStateBackend(new 
FsStateBackend("hdfs://namenode:40010/flink/checkpoints"))
+{% endhighlight %}
+</div>
+</div>
+
+
+### Setting Default State Backend
+
+A default state backend can be configured in the `flink-conf.yaml`, using the 
configuration key `state.backend`.
+
+Possible values for the config entry are *jobmanager* (MemoryStateBackend), 
*filesystem* (FsStateBackend), *rocksdb* (RocksDBStateBackend), or the fully 
qualified class
+name of the class that implements the state backend factory 
[FsStateBackendFactory](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java),
+such as `org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory` 
for RocksDBStateBackend.
+
+In the case where the default state backend is set to *filesystem*, the entry 
`state.backend.fs.checkpointdir` defines the directory where the checkpoint 
data will be stored.
+
+A sample section in the configuration file could look as follows:
+
+~~~
+# The backend that will be used to store operator state checkpoints
+
+state.backend: filesystem
+
+
+# Directory for storing checkpoints
+
+state.backend.fs.checkpointdir: hdfs://namenode:40010/flink/checkpoints
+~~~

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/state_backends.md
----------------------------------------------------------------------
diff --git a/docs/ops/state_backends.md b/docs/ops/state_backends.md
deleted file mode 100644
index 165cbad..0000000
--- a/docs/ops/state_backends.md
+++ /dev/null
@@ -1,169 +0,0 @@
----
-title: "State Backends"
-nav-parent_id: setup
-nav-pos: 11
----
-<!--
-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
-
-  http://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.
--->
-
-Programs written in the [Data Stream API]({{ site.baseurl 
}}/dev/datastream_api.html) often hold state in various forms:
-
-- Windows gather elements or aggregates until they are triggered
-- Transformation functions may use the key/value state interface to store 
values
-- Transformation functions may implement the `Checkpointed` interface to make 
their local variables fault tolerant
-
-See also [Working with State]({{ site.baseurl }}/dev/stream/state.html) in the 
streaming API guide.
-
-When checkpointing is activated, such state is persisted upon checkpoints to 
guard against data loss and recover consistently.
-How the state is represented internally, and how and where it is persisted 
upon checkpoints depends on the
-chosen **State Backend**.
-
-* ToC
-{:toc}
-
-## Available State Backends
-
-Out of the box, Flink bundles these state backends:
-
- - *MemoryStateBackend*
- - *FsStateBackend*
- - *RocksDBStateBackend*
-
-If nothing else is configured, the system will use the MemoryStateBackend.
-
-
-### The MemoryStateBackend
-
-The *MemoryStateBackend* holds data internally as objects on the Java heap. 
Key/value state and window operators hold hash tables
-that store the values, triggers, etc.
-
-Upon checkpoints, this state backend will snapshot the state and send it as 
part of the checkpoint acknowledgement messages to the
-JobManager (master), which stores it on its heap as well.
-
-The MemoryStateBackend can be configured to use asynchronous snapshots. While 
we strongly encourage the use of asynchronous snapshots to avoid blocking 
pipelines, please note that this is a new feature and currently not enabled 
-by default. To enable this feature, users can instantiate a 
`MemoryStateBackend` with the corresponding boolean flag in the constructor set 
to `true`, e.g.:
-
-{% highlight java %}
-    new MemoryStateBackend(MAX_MEM_STATE_SIZE, true);
-{% endhighlight %}
-
-Limitations of the MemoryStateBackend:
-
-  - The size of each individual state is by default limited to 5 MB. This 
value can be increased in the constructor of the MemoryStateBackend.
-  - Irrespective of the configured maximal state size, the state cannot be 
larger than the akka frame size (see [Configuration]({{ site.baseurl 
}}/setup/config.html)).
-  - The aggregate state must fit into the JobManager memory.
-
-The MemoryStateBackend is encouraged for:
-
-  - Local development and debugging
-  - Jobs that do hold little state, such as jobs that consist only of 
record-at-a-time functions (Map, FlatMap, Filter, ...). The Kafka Consumer 
requires very little state.
-
-
-### The FsStateBackend
-
-The *FsStateBackend* is configured with a file system URL (type, address, 
path), such as "hdfs://namenode:40010/flink/checkpoints" or 
"file:///data/flink/checkpoints".
-
-The FsStateBackend holds in-flight data in the TaskManager's memory. Upon 
checkpointing, it writes state snapshots into files in the configured file 
system and directory. Minimal metadata is stored in the JobManager's memory 
(or, in high-availability mode, in the metadata checkpoint).
-
-The FsStateBackend can be configured to use asynchronous snapshots. While we 
strongly encourage the use of asynchronous snapshots to avoid blocking 
pipelines, please note that this is a new feature and currently not enabled 
-by default. To enable this feature, users can instantiate a `FsStateBackend` 
with the corresponding boolean flag in the constructor set to `true`, e.g.:
-
-{% highlight java %}
-    new FsStateBackend(path, true);
-{% endhighlight %}
-
-The FsStateBackend is encouraged for:
-
-  - Jobs with large state, long windows, large key/value states.
-  - All high-availability setups.
-
-### The RocksDBStateBackend
-
-The *RocksDBStateBackend* is configured with a file system URL (type, address, 
path), such as "hdfs://namenode:40010/flink/checkpoints" or 
"file:///data/flink/checkpoints".
-
-The RocksDBStateBackend holds in-flight data in a 
[RocksDB](http://rocksdb.org) data base
-that is (per default) stored in the TaskManager data directories. Upon 
checkpointing, the whole
-RocksDB data base will be checkpointed into the configured file system and 
directory. Minimal
-metadata is stored in the JobManager's memory (or, in high-availability mode, 
in the metadata checkpoint).
-
-The RocksDBStateBackend always performs asynchronous snapshots.
-
-Limitations of the RocksDBStateBackend:
-
-  - As RocksDB's JNI bridge API is based on byte[], the maximum supported size 
per key and per value is 2^31 bytes each. 
-  IMPORTANT: states that use merge operations in RocksDB (e.g. ListState) can 
silently accumulate value sizes > 2^31 bytes and will then fail on their next 
retrieval. This is currently a limitation of RocksDB JNI.
-
-The RocksDBStateBackend is encouraged for:
-
-  - Jobs with very large state, long windows, large key/value states.
-  - All high-availability setups.
-
-Note that the amount of state that you can keep is only limited by the amount 
of disc space available.
-This allows keeping very large state, compared to the FsStateBackend that 
keeps state in memory.
-This also means, however, that the maximum throughput that can be achieved 
will be lower with
-this state backend.
-
-RocksDBStateBackend is currently the only backend that offers incremental 
checkpoints (see [here]({{ site.baseurl 
}}/monitoring/large_state_tuning.html)). 
-
-## Configuring a State Backend
-
-The default state backend, if you specify nothing, is the jobmanager. If you 
wish to establish a different default for all jobs on your cluster, you can do 
so by defining a new default state backend in **flink-conf.yaml**. The default 
state backend can be overridden on a per-job basis, as shown below.
-
-### Setting the Per-job State Backend
-
-The per-job state backend is set on the `StreamExecutionEnvironment` of the 
job, as shown in the example below:
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.setStateBackend(new 
FsStateBackend("hdfs://namenode:40010/flink/checkpoints"));
-{% endhighlight %}
-</div>
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val env = StreamExecutionEnvironment.getExecutionEnvironment()
-env.setStateBackend(new 
FsStateBackend("hdfs://namenode:40010/flink/checkpoints"))
-{% endhighlight %}
-</div>
-</div>
-
-
-### Setting Default State Backend
-
-A default state backend can be configured in the `flink-conf.yaml`, using the 
configuration key `state.backend`.
-
-Possible values for the config entry are *jobmanager* (MemoryStateBackend), 
*filesystem* (FsStateBackend), *rocksdb* (RocksDBStateBackend), or the fully 
qualified class
-name of the class that implements the state backend factory 
[FsStateBackendFactory](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java),
-such as `org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory` 
for RocksDBStateBackend.
-
-In the case where the default state backend is set to *filesystem*, the entry 
`state.backend.fs.checkpointdir` defines the directory where the checkpoint 
data will be stored.
-
-A sample section in the configuration file could look as follows:
-
-~~~
-# The backend that will be used to store operator state checkpoints
-
-state.backend: filesystem
-
-
-# Directory for storing checkpoints
-
-state.backend.fs.checkpointdir: hdfs://namenode:40010/flink/checkpoints
-~~~

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/ops/upgrading.md
----------------------------------------------------------------------
diff --git a/docs/ops/upgrading.md b/docs/ops/upgrading.md
index 6cfef81..12d15ea 100644
--- a/docs/ops/upgrading.md
+++ b/docs/ops/upgrading.md
@@ -1,6 +1,6 @@
 ---
 title: "Upgrading Applications and Flink Versions"
-nav-parent_id: setup
+nav-parent_id: ops
 nav-pos: 15
 ---
 <!--
@@ -31,7 +31,7 @@ This document describes how to update a Flink streaming 
application and how to m
 
 ## Restarting Streaming Applications
 
-The line of action for upgrading a streaming application or migrating an 
application to a different cluster is based on Flink's [Savepoint]({{ 
site.baseurl }}/setup/savepoints.html) feature. A savepoint is a consistent 
snapshot of the state of an application at a specific point in time. 
+The line of action for upgrading a streaming application or migrating an 
application to a different cluster is based on Flink's [Savepoint]({{ 
site.baseurl }}/ops/state/savepoints.html) feature. A savepoint is a consistent 
snapshot of the state of an application at a specific point in time. 
 
 There are two ways of taking a savepoint from a running streaming application.
 
@@ -83,7 +83,7 @@ When upgrading an application, user functions and operators 
can be freely modifi
 
 Operator state can be either user-defined or internal. 
 
-* **User-defined operator state:** In functions with user-defined operator 
state the type of the state is explicitly defined by the user. Although it is 
not possible to change the data type of operator state, a workaround to 
overcome this limitation can be to define a second state with a different data 
type and to implement logic to migrate the state from the original state into 
the new state. This approach requires a good migration strategy and a solid 
understanding of the behavior of [key-partitioned state]({{ site.baseurl 
}}/dev/stream/state.html).
+* **User-defined operator state:** In functions with user-defined operator 
state the type of the state is explicitly defined by the user. Although it is 
not possible to change the data type of operator state, a workaround to 
overcome this limitation can be to define a second state with a different data 
type and to implement logic to migrate the state from the original state into 
the new state. This approach requires a good migration strategy and a solid 
understanding of the behavior of [key-partitioned state]({{ site.baseurl 
}}/dev/stream/state/state.html).
 
 * **Internal operator state:** Operators such as window or join operators hold 
internal operator state which is not exposed to the user. For these operators 
the data type of the internal state depends on the input or output type of the 
operator. Consequently, changing the respective input or output type breaks 
application state consistency and prevents an upgrade. The following table 
lists operators with internal state and shows how the state data type relates 
to their input and output types. For operators which are applied on a keyed 
stream, the key type (KEY) is always part of the state data type as well.
 
@@ -141,7 +141,7 @@ about the steps that we outlined before.
 ### Preconditions
 
 Before starting the migration, please check that the jobs you are trying to 
migrate are following the
-best practises for [savepoints]({{ site.baseurl }}/setup/savepoints.html). 
Also, check out the 
+best practises for [savepoints]({{ site.baseurl }}/ops/state/savepoints.html). 
Also, check out the 
 [API Migration Guides]({{ site.baseurl }}/dev/migration.html) to see if there 
is any API changes related to migrating
 savepoints to newer versions.
 
@@ -175,7 +175,7 @@ You can do this with the command:
 $ bin/flink savepoint :jobId [:targetDirectory]
 ```
 
-For more details, please read the [savepoint documentation]({{ site.baseurl 
}}/setup/savepoints.html).
+For more details, please read the [savepoint documentation]({{ site.baseurl 
}}/ops/state/savepoints.html).
 
 ### STEP 2: Update your cluster to the new Flink version.
 
@@ -183,7 +183,7 @@ In this step, we update the framework version of the 
cluster. What this basicall
 the Flink installation with the new version. This step can depend on how you 
are running Flink in your cluster (e.g. 
 standalone, on Mesos, ...).
 
-If you are unfamiliar with installing Flink in your cluster, please read the 
[deployment and cluster setup documentation]({{ site.baseurl 
}}/setup/index.html).
+If you are unfamiliar with installing Flink in your cluster, please read the 
[deployment and cluster setup documentation]({{ site.baseurl 
}}/ops/deployment/cluster_setup.html).
 
 ### STEP 3: Resume the job under the new Flink version from savepoint.
 
@@ -194,7 +194,7 @@ this with the command:
 $ bin/flink run -s :savepointPath [:runArgs]
 ```
 
-Again, for more details, please take a look at the [savepoint 
documentation]({{ site.baseurl }}/setup/savepoints.html).
+Again, for more details, please take a look at the [savepoint 
documentation]({{ site.baseurl }}/ops/state/savepoints.html).
 
 ## Compatibility Table
 

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/quickstart/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/setup_quickstart.md 
b/docs/quickstart/setup_quickstart.md
index 7800dec..1b5a1e9 100644
--- a/docs/quickstart/setup_quickstart.md
+++ b/docs/quickstart/setup_quickstart.md
@@ -30,7 +30,7 @@ Get a Flink example program up and running in a few simple 
steps.
 
 ## Setup: Download and Start Flink
 
-Flink runs on __Linux, Mac OS X, and Windows__. To be able to run Flink, the 
only requirement is to have a working __Java 8.x__ (or higher) installation. 
Windows users, please take a look at the [Flink on Windows]({{ site.baseurl 
}}/setup/flink_on_windows.html) guide which describes how to run Flink on 
Windows for local setups.
+Flink runs on __Linux, Mac OS X, and Windows__. To be able to run Flink, the 
only requirement is to have a working __Java 7.x__ (or higher) installation. 
Windows users, please take a look at the [Flink on Windows]({{ site.baseurl 
}}/start/flink_on_windows.html) guide which describes how to run Flink on 
Windows for local setups.
 
 You can check the correct installation of Java by issuing the following 
command:
 

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/redirects/cli.md
----------------------------------------------------------------------
diff --git a/docs/redirects/cli.md b/docs/redirects/cli.md
index eb7777d..59abf0d 100644
--- a/docs/redirects/cli.md
+++ b/docs/redirects/cli.md
@@ -1,7 +1,7 @@
 ---
 title: "CLI"
 layout: redirect
-redirect: /setup/cli.html
+redirect: /ops/cli.html
 permalink: /apis/cli.html
 ---
 <!--

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/redirects/fault_tolerance.md
----------------------------------------------------------------------
diff --git a/docs/redirects/fault_tolerance.md 
b/docs/redirects/fault_tolerance.md
index b68ecd7..7a5060b 100644
--- a/docs/redirects/fault_tolerance.md
+++ b/docs/redirects/fault_tolerance.md
@@ -1,7 +1,7 @@
 ---
 title: "Fault Tolerance"
 layout: redirect
-redirect: /dev/stream/state.html
+redirect: /dev/stream/state/checkpointing.html
 permalink: /apis/streaming/fault_tolerance.html
 ---
 <!--

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/redirects/savepoints.md
----------------------------------------------------------------------
diff --git a/docs/redirects/savepoints.md b/docs/redirects/savepoints.md
index 2863d74..744b169 100644
--- a/docs/redirects/savepoints.md
+++ b/docs/redirects/savepoints.md
@@ -1,7 +1,7 @@
 ---
 title: "Savepoints"
 layout: redirect
-redirect: /setup/savepoints.html
+redirect: /ops/state/savepoints.html
 permalink: /apis/streaming/savepoints.html
 ---
 <!--

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/redirects/state.md
----------------------------------------------------------------------
diff --git a/docs/redirects/state.md b/docs/redirects/state.md
index 15869ba..0f5784f 100644
--- a/docs/redirects/state.md
+++ b/docs/redirects/state.md
@@ -1,7 +1,7 @@
 ---
 title: "Working with State"
 layout: redirect
-redirect: /dev/stream/state.html
+redirect: /dev/stream/state/state.html
 permalink: /apis/streaming/state.html
 ---
 <!--

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/redirects/state_backends.md
----------------------------------------------------------------------
diff --git a/docs/redirects/state_backends.md b/docs/redirects/state_backends.md
index 3a21aaa..e0b462d 100644
--- a/docs/redirects/state_backends.md
+++ b/docs/redirects/state_backends.md
@@ -1,7 +1,7 @@
 ---
 title: "State Backends"
 layout: redirect
-redirect: /ops/state_backends.html
+redirect: /ops/state/state_backends.html
 permalink: /apis/streaming/state_backends.html
 ---
 <!--

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/setup/aws.md
----------------------------------------------------------------------
diff --git a/docs/setup/aws.md b/docs/setup/aws.md
deleted file mode 100644
index bc5b221..0000000
--- a/docs/setup/aws.md
+++ /dev/null
@@ -1,374 +0,0 @@
----
-title: "Amazon Web Services (AWS)"
-nav-title: AWS
-nav-parent_id: deployment
-nav-pos: 5
----
-<!--
-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
-
-  http://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.
--->
-
-Amazon Web Services offers cloud computing services on which you can run Flink.
-
-* ToC
-{:toc}
-
-## EMR: Elastic MapReduce
-
-[Amazon Elastic MapReduce](https://aws.amazon.com/elasticmapreduce/) (Amazon 
EMR) is a web service that makes it easy to  quickly setup a Hadoop cluster. 
This is the **recommended way** to run Flink on AWS as it takes care of setting 
up everything.
-
-### Standard EMR Installation
-
-Flink is a supported application on Amazon EMR. [Amazon's 
documentation](http://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html)
-describes configuring Flink, creating and monitoring a cluster, and working 
with jobs.
-
-### Custom EMR Installation
-
-Amazon EMR services are regularly updated to new releases but a version of 
Flink which is not available
-can be manually installed in a stock EMR cluster.
-
-**Create EMR Cluster**
-
-The EMR documentation contains [examples showing how to start an EMR 
cluster](http://docs.aws.amazon.com/ElasticMapReduce/latest/ManagementGuide/emr-gs-launch-sample-cluster.html).
 You can follow that guide and install any EMR release. You don't need to 
install the *All Applications* part of the EMR release, but can stick to *Core 
Hadoop*.
-
-{% warn Note %}
-Access to S3 buckets requires
-[configuration of IAM 
roles](http://docs.aws.amazon.com/ElasticMapReduce/latest/ManagementGuide/emr-iam-roles.html)
-when creating an EMR cluster.
-
-**Install Flink on EMR Cluster**
-
-After creating your cluster, you can [connect to the master 
node](http://docs.aws.amazon.com/ElasticMapReduce/latest/ManagementGuide/emr-connect-master-node.html)
 and install Flink:
-
-1. Go the the [Downloads Page]({{ download_url}}) and **download a binary 
version of Flink matching the Hadoop version** of your EMR cluster, e.g. Hadoop 
2.7 for EMR releases 4.3.0, 4.4.0, or 4.5.0.
-2. Extract the Flink distribution and you are ready to deploy [Flink jobs via 
YARN]({{ site.baseurl }}/setup/yarn_setup.html) after **setting the Hadoop 
config directory**:
-
-```bash
-HADOOP_CONF_DIR=/etc/hadoop/conf bin/flink run -m yarn-cluster 
examples/streaming/WordCount.jar
-```
-
-{% top %}
-
-## S3: Simple Storage Service
-
-[Amazon Simple Storage Service](http://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_backends.html).
-
-You can use S3 objects like regular files by specifying paths in the following 
format:
-
-```
-s3://<your-bucket>/<endpoint>
-```
-
-The endpoint can either be a single file or a directory, for example:
-
-```java
-// Read from S3 bucket
-env.readTextFile("s3://<bucket>/<endpoint>");
-
-// Write to S3 bucket
-stream.writeAsText("s3://<bucket>/<endpoint>");
-
-// Use S3 as FsStatebackend
-env.setStateBackend(new FsStateBackend("s3://<your-bucket>/<endpoint>"));
-```
-
-Note that these examples are *not* exhaustive and you can use S3 in other 
places as well, including your [high availability setup]({{ site.baseurl 
}}/setup/jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ 
site.baseurl }}/ops/state_backends.html#the-rocksdbstatebackend);  everywhere 
that Flink expects a FileSystem URI.
-
-### Set S3 FileSystem
-
-{% panel **Note:** You don't have to configure this manually if you are 
running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-S3 is treated by Flink as a regular FileSystem. Interaction with S3 happens 
via a Hadoop [S3 FileSystem client](https://wiki.apache.org/hadoop/AmazonS3).
-
-There are two popular S3 file system implementations available:
-
-1. `S3AFileSystem` (**recommended**): file system for reading and writing 
regular files using Amazon's SDK internally. No maximum file size and works 
with IAM roles.
-2. `NativeS3FileSystem`: file system for reading and writing regular files. 
Maximum object size is 5GB and does not work with IAM roles.
-
-#### `S3AFileSystem` (Recommended)
-
-This is the recommended S3 FileSystem implementation to use. It uses Amazon's 
SDK internally and works with IAM roles (see [Configure Access 
Credential](#configure-access-credentials)).
-
-You need to point Flink to a valid Hadoop configuration, which contains the 
following properties in `core-site.xml`:
-
-```xml
-<configuration>
-
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-</property>
-
-<!-- Comma separated list of local directories used to buffer
-     large results prior to transmitting them to S3. -->
-<property>
-  <name>fs.s3.buffer.dir</name>
-  <value>/tmp</value>
-</property>
-
-</configuration>
-```
-
-This registers `S3AFileSystem` as the default FileSystem for URIs with the 
`s3://` scheme.
-
-#### `NativeS3FileSystem`
-
-This file system is limited to files up to 5GB in size and it does not work 
IAM roles (see [Configure Access Credential](#configure-access-credentials)), 
meaning that you have to manually configure your AWS credentials in the Hadoop 
config file.
-
-You need to point Flink to a valid Hadoop configuration, which contains the 
following property in `core-site.xml`:
-
-```xml
-<property>
-  <name>fs.s3.impl</name>
-  <value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value>
-</property>
-```
-
-This registers `NativeS3FileSystem` as the default FileSystem for URIs with 
the `s3://` scheme.
-
-#### Hadoop Configuration
-
-You can specify the [Hadoop configuration]({{ site.baseurl 
}}/setup/config.html#hdfs) in various ways, for example by configuring the path 
to the Hadoop configuration directory in `flink-conf.yaml`:
-
-```
-fs.hdfs.hadoopconf: /path/to/etc/hadoop
-```
-
-This registers `/path/to/etc/hadoop` as Hadoop's configuration directory with 
Flink. Flink will look for the `core-site.xml` and `hdfs-site.xml` files in the 
specified directory.  
-
-{% top %}
-
-### Configure Access Credentials
-
-{% panel **Note:** You don't have to configure this manually if you are 
running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-After setting up the S3 FileSystem, you need to make sure that Flink is 
allowed to access your S3 buckets.
-
-#### Identity and Access Management (IAM) (Recommended)
-
-When using `S3AFileSystem` 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 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](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.
-
-Note that this only works with `S3AFileSystem` and not `NativeS3FileSystem`.
-
-{% top %}
-
-#### Access Keys with S3AFileSystem (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).
-
-For `S3AFileSystem` you need to configure both `fs.s3a.access.key` and 
`fs.s3a.secret.key`  in Hadoop's  `core-site.xml`:
-
-```xml
-<property>
-  <name>fs.s3a.access.key</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <value></value>
-</property>
-```
-
-{% top %}
-
-#### Access Keys with NativeS3FileSystem (Discouraged)
-
-Access to S3 can be granted via your **access and secret key pair**. But this 
is discouraged and you should use `S3AFileSystem` [with the required IAM 
roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2).
-
-For `NativeS3FileSystem` you need to configure both `fs.s3.awsAccessKeyId` and 
`fs.s3.awsSecretAccessKey`  in Hadoop's  `core-site.xml`:
-
-```xml
-<property>
-  <name>fs.s3.awsAccessKeyId</name>
-  <value></value>
-</property>
-
-<property>
-  <name>fs.s3.awsSecretAccessKey</name>
-  <value></value>
-</property>
-```
-
-{% top %}
-
-### Provide S3 FileSystem Dependency
-
-{% panel **Note:** You don't have to configure this manually if you are 
running [Flink on EMR](#emr-elastic-mapreduce). %}
-
-Hadoop's S3 FileSystem clients are packaged in the `hadoop-aws`. This JAR and 
all its dependencies need to be added to Flink's classpath, i.e. the class path 
of both Job and TaskManagers. Depending on which FileSystem implementation and 
which Flink and Hadoop version you use, you need to provide different 
dependencies (see below).
-
-There are multiple ways of adding JARs to Flink's class path, the easiest 
being simply to drop the JARs in Flink's `/lib` folder. You need to copy the 
`hadoop-aws` JAR with all its dependencies. You can also export the directory 
containing these JARs as part of the `HADOOP_CLASSPATH` environment variable on 
all machines.
-
-#### Flink for Hadoop 2.7
-
-Depending on which file system you use, please add the following dependencies. 
You can find these as part of the Hadoop binaries in 
`hadoop-2.7/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.7.2.jar`
-  - `aws-java-sdk-1.7.4.jar`
-  - `httpcore-4.2.5.jar`
-  - `httpclient-4.2.5.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.7.2.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded 
by Flink.
-
-#### Flink for Hadoop 2.6
-
-Depending on which file system you use, please add the following dependencies. 
You can find these as part of the Hadoop binaries in 
`hadoop-2.6/share/hadoop/tools/lib`:
-
-- `S3AFileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `aws-java-sdk-1.7.4.jar`
-  - `httpcore-4.2.5.jar`
-  - `httpclient-4.2.5.jar`
-
-- `NativeS3FileSystem`:
-  - `hadoop-aws-2.6.4.jar`
-  - `guava-11.0.2.jar`
-
-Note that `hadoop-common` is available as part of Flink, but Guava is shaded 
by Flink.
-
-#### Flink for Hadoop 2.4 and earlier
-
-These Hadoop versions only have support for `NativeS3FileSystem`. This comes 
pre-packaged with Flink for Hadoop 2 as part of `hadoop-common`. You don't need 
to add anything to the classpath.
-
-{% top %}
-
-## Common Issues
-
-The following sections lists common issues when working with Flink on AWS.
-
-### Missing S3 FileSystem Configuration
-
-If your job submission fails with an Exception message noting that `No file 
system found with scheme s3` this means that no FileSystem has been configured 
for S3. Please check out the [FileSystem Configuration 
section](#set-s3-filesystem) for details on how to configure this properly.
-
-```
-org.apache.flink.client.program.ProgramInvocationException: The program 
execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) 
[...]
-Caused by: org.apache.flink.runtime.JobException: Creating the input splits 
caused an error:
-  No file system found with scheme s3, referenced in file URI 
's3://<bucket>/<endpoint>'. [...]
-Caused by: java.io.IOException: No file system found with scheme s3,
-  referenced in file URI 's3://<bucket>/<endpoint>'.
-    at o.a.f.core.fs.FileSystem.get(FileSystem.java:296)
-    at o.a.f.core.fs.Path.getFileSystem(Path.java:311)
-    at 
o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-    at 
o.a.f.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-    at 
o.a.f.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-```
-
-{% top %}
-
-### AWS Access Key ID and Secret Access Key Not Specified
-
-If you see your job failing with an Exception noting that the `AWS Access Key 
ID and Secret Access Key must be specified as the username or password`, your 
access credentials have not been set up properly. Please refer to the [access 
credential section](#configure-access-credentials) for details on how to 
configure this.
-
-```
-org.apache.flink.client.program.ProgramInvocationException: The program 
execution failed:
-  Failed to submit job cd927567a81b62d7da4c18eaa91c3c39 (WordCount Example) 
[...]
-Caused by: java.io.IOException: The given file URI (s3://<bucket>/<endpoint>) 
points to the
-  HDFS NameNode at <bucket>, but the File System could not be initialized with 
that address:
-  AWS Access Key ID and Secret Access Key must be specified as the username or 
password
-  (respectively) of a s3n URL, or by setting the fs.s3n.awsAccessKeyId
-  or fs.s3n.awsSecretAccessKey properties (respectively) [...]
-Caused by: java.lang.IllegalArgumentException: AWS Access Key ID and Secret 
Access Key must
-  be specified as the username or password (respectively) of a s3 URL, or by 
setting
-  the fs.s3n.awsAccessKeyId or fs.s3n.awsSecretAccessKey properties 
(respectively) [...]
-    at o.a.h.fs.s3.S3Credentials.initialize(S3Credentials.java:70)
-    at 
o.a.h.fs.s3native.Jets3tNativeFileSystemStore.initialize(Jets3tNativeFileSystemStore.java:80)
-    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
-    at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
-    at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
-    at java.lang.reflect.Method.invoke(Method.java:606)
-    at 
o.a.h.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
-    at 
o.a.h.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
-    at o.a.h.fs.s3native.$Proxy6.initialize(Unknown Source)
-    at 
o.a.h.fs.s3native.NativeS3FileSystem.initialize(NativeS3FileSystem.java:330)
-    at 
o.a.f.runtime.fs.hdfs.HadoopFileSystem.initialize(HadoopFileSystem.java:321)
-```
-
-{% top %}
-
-### ClassNotFoundException: NativeS3FileSystem/S3AFileSystem Not Found
-
-If you see this Exception, the S3 FileSystem is not part of the class path of 
Flink. Please refer to [S3 FileSystem dependency 
section](#provide-s3-filesystem-dependency) for details on how to configure 
this properly.
-
-```
-Caused by: java.lang.RuntimeException: java.lang.RuntimeException: 
java.lang.ClassNotFoundException: Class 
org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2186)
-  at 
org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.getHadoopWrapperClassNameForFileSystem(HadoopFileSystem.java:460)
-  at 
org.apache.flink.core.fs.FileSystem.getHadoopWrapperClassNameForFileSystem(FileSystem.java:352)
-  at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:280)
-  at org.apache.flink.core.fs.Path.getFileSystem(Path.java:311)
-  at 
org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:450)
-  at 
org.apache.flink.api.common.io.FileInputFormat.createInputSplits(FileInputFormat.java:57)
-  at 
org.apache.flink.runtime.executiongraph.ExecutionJobVertex.<init>(ExecutionJobVertex.java:156)
-  ... 25 more
-Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class 
org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2154)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2178)
-  ... 32 more
-Caused by: java.lang.ClassNotFoundException: Class 
org.apache.hadoop.fs.s3native.NativeS3FileSystem not found
-  at 
org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2060)
-  at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2152)
-  ... 33 more
-```
-
-{% top %}
-
-### IOException: `400: Bad Request`
-
-If you you have configured everything properly, but get a `Bad Request` 
Exception **and** your S3 bucket is located in region `eu-central-1`, you might 
be running an S3 client, which does not support [Amazon's signature version 
4](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
-
-Currently, this includes all Hadoop versions up to 2.7.2 running 
`NativeS3FileSystem`, which depend on `JetS3t 0.9.0` instead of a version [>= 
0.9.4](http://www.jets3t.org/RELEASE_NOTES.html).
-
-The only workaround is to change the bucket region.
-
-```
-[...]
-Caused by: java.io.IOException: s3://<bucket-in-eu-central-1>/<endpoint> : 400 
: Bad Request [...]
-Caused by: org.jets3t.service.impl.rest.HttpException [...]
-```
-
-{% top %}
-
-### NullPointerException at org.apache.hadoop.fs.LocalDirAllocator
-
-This Exception is usually caused by skipping the local buffer directory 
configuration `fs.s3.buffer.dir` for the `S3AFileSystem`. Please refer to the 
[S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to 
configure the `S3AFileSystem` properly.
-
-```
-[...]
-Caused by: java.lang.NullPointerException at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:268)
 at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:344)
 at
-o.a.h.fs.LocalDirAllocator$AllocatorPerContext.createTmpFileForWrite(LocalDirAllocator.java:416)
 at
-o.a.h.fs.LocalDirAllocator.createTmpFileForWrite(LocalDirAllocator.java:198) at
-o.a.h.fs.s3a.S3AOutputStream.<init>(S3AOutputStream.java:87) at
-o.a.h.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:410) at
-o.a.h.fs.FileSystem.create(FileSystem.java:907) at
-o.a.h.fs.FileSystem.create(FileSystem.java:888) at
-o.a.h.fs.FileSystem.create(FileSystem.java:785) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:404) at
-o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:48) at
-... 25 more
-```

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/setup/building.md
----------------------------------------------------------------------
diff --git a/docs/setup/building.md b/docs/setup/building.md
deleted file mode 100644
index 01dc4ed..0000000
--- a/docs/setup/building.md
+++ /dev/null
@@ -1,149 +0,0 @@
----
-title: Building Flink from Source
-nav-parent_id: start
-nav-pos: 20
----
-<!--
-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
-
-  http://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.
--->
-
-This page covers how to build Flink {{ site.version }} from sources.
-
-* This will be replaced by the TOC
-{:toc}
-
-## Build Flink
-
-In order to build Flink you need the source code. Either [download the source 
of a release]({{ site.download_url }}) or [clone the git repository]({{ 
site.github_url }}).
-
-In addition you need **Maven 3** and a **JDK** (Java Development Kit). Flink 
requires **at least Java 8** to build.
-
-*NOTE: Maven 3.3.x can build Flink, but will not properly shade away certain 
dependencies. Maven 3.0.3 creates the libraries properly.
-To build unit tests use Java 8u51 or above to prevent failures in unit tests 
that use the PowerMock runner.*
-
-To clone from git, enter:
-
-~~~bash
-git clone {{ site.github_url }}
-~~~
-
-The simplest way of building Flink is by running:
-
-~~~bash
-mvn clean install -DskipTests
-~~~
-
-This instructs [Maven](http://maven.apache.org) (`mvn`) to first remove all 
existing builds (`clean`) and then create a new Flink binary (`install`). The 
`-DskipTests` command prevents Maven from executing the tests.
-
-The default build includes the YARN Client for Hadoop 2.
-
-## Dependency Shading
-
-Flink [shades away](https://maven.apache.org/plugins/maven-shade-plugin/) some 
of the libraries it uses, in order to avoid version clashes with user programs 
that use different versions of these libraries. Among the shaded libraries are 
*Google Guava*, *Asm*, *Apache Curator*, *Apache HTTP Components*, and others.
-
-The dependency shading mechanism was recently changed in Maven and requires 
users to build Flink slightly differently, depending on their Maven version:
-
-**Maven 3.0.x, 3.1.x, and 3.2.x**
-It is sufficient to call `mvn clean install -DskipTests` in the root directory 
of Flink code base.
-
-**Maven 3.3.x**
-The build has to be done in two steps: First in the base directory, then in 
the distribution project:
-
-~~~bash
-mvn clean install -DskipTests
-cd flink-dist
-mvn clean install
-~~~
-
-*Note:* To check your Maven version, run `mvn --version`.
-
-{% top %}
-
-## Hadoop Versions
-
-{% info %} Most users do not need to do this manually. The [download page]({{ 
site.download_url }}) contains binary packages for common Hadoop versions.
-
-Flink has dependencies to HDFS and YARN which are both dependencies from 
[Apache Hadoop](http://hadoop.apache.org). There exist many different versions 
of Hadoop (from both the upstream project and the different Hadoop 
distributions). If you are using a wrong combination of versions, exceptions 
can occur.
-
-Hadoop is only supported from version 2.3.0 upwards.
-You can also specify a specific Hadoop version to build against:
-
-~~~bash
-mvn clean install -DskipTests -Dhadoop.version=2.6.1
-~~~
-
-#### Before Hadoop 2.3.0
-
-Hadoop 2.x versions are only supported with YARN features from version 2.3.0 
upwards. If you want to use a version lower than 2.3.0, you can exclude the 
YARN support using the following extra build arguments: `-P!include-yarn`.
-
-For example, if you want to build Flink for Hadoop `2.2.0`, use the following 
command:
-
-~~~bash
-mvn clean install -Dhadoop.version=2.2.0 -P!include-yarn
-~~~
-
-### Vendor-specific Versions
-
-To build Flink against a vendor specific Hadoop version, issue the following 
command:
-
-~~~bash
-mvn clean install -DskipTests -Pvendor-repos -Dhadoop.version=2.6.1-cdh5.0.0
-~~~
-
-The `-Pvendor-repos` activates a Maven [build 
profile](http://maven.apache.org/guides/introduction/introduction-to-profiles.html)
 that includes the repositories of popular Hadoop vendors such as Cloudera, 
Hortonworks, or MapR.
-
-{% top %}
-
-## Scala Versions
-
-{% info %} Users that purely use the Java APIs and libraries can *ignore* this 
section.
-
-Flink has APIs, libraries, and runtime modules written in 
[Scala](http://scala-lang.org). Users of the Scala API and libraries may have 
to match the Scala version of Flink with the Scala version of their projects 
(because Scala is not strictly backwards compatible).
-
-**By default, Flink is built with the Scala 2.11**. To build Flink with Scala 
*2.10*, you can change the default Scala *binary version* by using *scala-2.10* 
build profile:
-
-~~~bash
-# Build with Scala version 2.10
-mvn clean install -DskipTests -Pscala-2.10
-~~~
-
-To build against custom Scala versions, you need to define new custom build 
profile that will override *scala.version* and *scala.binary.version* values.
-
-Flink is developed against Scala *2.11* and tested additionally against Scala 
*2.10*. These two versions are known to be compatible. Earlier versions (like 
Scala *2.9*) are *not* compatible.
-
-Newer versions may be compatible, depending on breaking changes in the 
language features used by Flink, and the availability of Flink's dependencies 
in those Scala versions. The dependencies written in Scala include for example 
*Kafka*, *Akka*, *Scalatest*, and *scopt*.
-
-{% top %}
-
-## Encrypted File Systems
-
-If your home directory is encrypted you might encounter a 
`java.io.IOException: File name too long` exception. Some encrypted file 
systems, like encfs used by Ubuntu, do not allow long filenames, which is the 
cause of this error.
-
-The workaround is to add:
-
-~~~xml
-<args>
-    <arg>-Xmax-classfile-name</arg>
-    <arg>128</arg>
-</args>
-~~~
-
-in the compiler configuration of the `pom.xml` file of the module causing the 
error. For example, if the error appears in the `flink-yarn` module, the above 
code should be added under the `<configuration>` tag of `scala-maven-plugin`. 
See [this issue](https://issues.apache.org/jira/browse/FLINK-2003) for more 
information.
-
-{% top %}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/47070674/docs/setup/checkpoints.md
----------------------------------------------------------------------
diff --git a/docs/setup/checkpoints.md b/docs/setup/checkpoints.md
deleted file mode 100644
index 0070cb1..0000000
--- a/docs/setup/checkpoints.md
+++ /dev/null
@@ -1,101 +0,0 @@
----
-title: "Checkpoints"
-nav-parent_id: setup
-nav-pos: 7
----
-<!--
-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
-
-  http://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.
--->
-
-
-* toc
-{:toc}
-
-## Overview
-
-Checkpoints make state in Flink fault tolerant by allowing state and the
-corresponding stream positions to be recovered, thereby giving the application
-the same semantics as a failure-free execution.
-
-See [Checkpointing](../dev/stream/checkpointing.html) for how to enable and
-configure checkpoints for your program.
-
-## Externalized Checkpoints
-
-Checkpoints are by default not persisted externally and are only used to
-resume a job from failures. They are deleted when a program is cancelled.
-You can, however, configure periodic checkpoints to be persisted externally
-similarly to [savepoints](savepoints.html). These *externalized checkpoints*
-write their meta data out to persistent storage and are *not* automatically
-cleaned up when the job fails. This way, you will have a checkpoint around
-to resume from if your job fails.
-
-```java
-CheckpointConfig config = env.getCheckpointConfig();
-config.enableExternalizedCheckpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
-```
-
-The `ExternalizedCheckpointCleanup` mode configures what happens with 
externalized checkpoints when you cancel the job:
-
-- **`ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION`**: Retain the 
externalized checkpoint when the job is cancelled. Note that you have to 
manually clean up the checkpoint state after cancellation in this case.
-
-- **`ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION`**: Delete the 
externalized checkpoint when the job is cancelled. The checkpoint state will 
only be available if the job fails.
-
-### Directory Structure
-
-Similarly to [savepoints](savepoints.html), an externalized checkpoint consists
-of a meta data file and, depending on the state back-end, some additional data
-files. The **target directory** for the externalized checkpoint's meta data is
-determined from the configuration key `state.checkpoints.dir` which, currently,
-can only be set via the configuration files.
-
-```
-state.checkpoints.dir: hdfs:///checkpoints/
-```
-
-This directory will then contain the checkpoint meta data required to restore
-the checkpoint. For the `MemoryStateBackend`, this meta data file will be
-self-contained and no further files are needed.
-
-`FsStateBackend` and `RocksDBStateBackend` write separate data files
-and only write the paths to these files into the meta data file. These data
-files are stored at the path given to the state back-end during construction.
-
-```java
-env.setStateBackend(new RocksDBStateBackend("hdfs:///checkpoints-data/");
-```
-
-### Difference to Savepoints
-
-Externalized checkpoints have a few differences from 
[savepoints](savepoints.html). They
-- use a state backend specific (low-level) data format,
-- may be incremental,
-- do not support Flink specific features like rescaling.
-
-### Resuming from an externalized checkpoint
-
-A job may be resumed from an externalized checkpoint just as from a savepoint
-by using the checkpoint's meta data file instead (see the
-[savepoint restore guide](cli.html#restore-a-savepoint)). Note that if the
-meta data file is not self-contained, the jobmanager needs to have access to
-the data files it refers to (see [Directory Structure](#directory-structure)
-above).
-
-```sh
-$ bin/flink run -s :checkpointMetaDataPath [:runArgs]
-```

Reply via email to