http://git-wip-us.apache.org/repos/asf/flink/blob/844c874b/docs/apis/streaming/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming/libs/cep.md b/docs/apis/streaming/libs/cep.md
deleted file mode 100644
index ef35d32..0000000
--- a/docs/apis/streaming/libs/cep.md
+++ /dev/null
@@ -1,659 +0,0 @@
----
-title: "FlinkCEP - Complex event processing for Flink"
-# Top navigation
-top-nav-group: libs
-top-nav-pos: 2
-top-nav-title: CEP
-# Sub navigation
-sub-nav-group: streaming
-sub-nav-id: cep
-sub-nav-pos: 1
-sub-nav-parent: libs
-sub-nav-title: Event Processing (CEP)
----
-<!--
-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.
--->
-
-FlinkCEP is the complex event processing library for Flink.
-It allows you to easily detect complex event patterns in a stream of endless 
data.
-Complex events can then be constructed from matching sequences.
-This gives you the opportunity to quickly get hold of what's really important 
in your data.
-
-<span class="label label-danger">Attention</span> The events in the 
`DataStream` to which
-you want to apply pattern matching have to implement proper `equals()` and 
`hashCode()` methods
-because these are used for comparing and matching events.
-
-* This will be replaced by the TOC
-{:toc}
-
-## Getting Started
-
-If you want to jump right in, you have to [set up a Flink program]({{ 
site.baseurl }}/apis/common/index.html#linking-with-flink).
-Next, you have to add the FlinkCEP dependency to the `pom.xml` of your project.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight xml %}
-<dependency>
-  <groupId>org.apache.flink</groupId>
-  <artifactId>flink-cep{{ site.scala_version_suffix }}</artifactId>
-  <version>{{site.version }}</version>
-</dependency>
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight xml %}
-<dependency>
-  <groupId>org.apache.flink</groupId>
-  <artifactId>flink-cep-scala{{ site.scala_version_suffix }}</artifactId>
-  <version>{{site.version }}</version>
-</dependency>
-{% endhighlight %}
-</div>
-</div>
-
-Note that FlinkCEP is currently not part of the binary distribution.
-See linking with it for cluster execution 
[here]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
-
-Now you can start writing your first CEP program using the pattern API.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-DataStream<Event> input = ...
-
-Pattern<Event, ?> pattern = Pattern.begin("start").where(evt -> evt.getId() == 
42)
-    .next("middle").subtype(SubEvent.class).where(subEvt -> subEvt.getVolume() 
>= 10.0)
-    .followedBy("end").where(evt -> evt.getName().equals("end"));
-
-PatternStream<Event> patternStream = CEP.pattern(input, pattern);
-
-DataStream<Alert> result = patternStream.select(pattern -> {
-    return createAlertFrom(pattern);
-});
-{% endhighlight %}
-</div>
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val input: DataStream[Event] = ...
-
-val pattern = Pattern.begin("start").where(_.getId == 42)
-  .next("middle").subtype(classOf[SubEvent]).where(_.getVolume >= 10.0)
-  .followedBy("end").where(_.getName == "end")
-
-val patternStream = CEP.pattern(input, pattern)
-
-val result: DataStream[Alert] = patternStream.select(createAlert(_))
-{% endhighlight %}
-</div>
-</div>
-
-Note that we use use Java 8 lambdas in our Java code examples to make them 
more succinct.
-
-## The Pattern API
-
-The pattern API allows you to quickly define complex event patterns.
-
-Each pattern consists of multiple stages or what we call states.
-In order to go from one state to the next, the user can specify conditions.
-These conditions can be the contiguity of events or a filter condition on an 
event.
-
-Each pattern has to start with an initial state:
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-Pattern<Event, ?> start = Pattern.<Event>begin("start");
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val start : Pattern[Event, _] = Pattern.begin("start")
-{% endhighlight %}
-</div>
-</div>
-
-Each state must have an unique name to identify the matched events later on.
-Additionally, we can specify a filter condition for the event to be accepted 
as the start event via the `where` method.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-start.where(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) {
-        return ... // some condition
-    }
-});
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-start.where(event => ... /* some condition */)
-{% endhighlight %}
-</div>
-</div>
-
-We can also restrict the type of the accepted event to some subtype of the 
initial event type (here `Event`) via the `subtype` method.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-start.subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
-    @Override
-    public boolean filter(SubEvent value) {
-        return ... // some condition
-    }
-});
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */)
-{% endhighlight %}
-</div>
-</div>
-
-As it can be seen here, the subtype condition can also be combined with an 
additional filter condition on the subtype.
-In fact you can always provide multiple conditions by calling `where` and 
`subtype` multiple times.
-These conditions will then be combined using the logical AND operator.
-
-In order to construct or conditions, one has to call the `or` method with a 
respective filter function.
-Any existing filter function is then ORed with the given one.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-pattern.where(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) {
-        return ... // some condition
-    }
-}).or(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) {
-        return ... // or condition
-    }
-});
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-pattern.where(event => ... /* some condition */).or(event => ... /* or 
condition */)
-{% endhighlight %}
-</div>
-</div>
-
-Next, we can append further states to detect complex patterns.
-We can control the contiguity of two succeeding events to be accepted by the 
pattern.
-
-Strict contiguity means that two matching events have to succeed directly.
-This means that no other events can occur in between.
-A strict contiguity pattern state can be created via the `next` method.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-Pattern<Event, ?> strictNext = start.next("middle");
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val strictNext: Pattern[Event, _] = start.next("middle")
-{% endhighlight %}
-</div>
-</div>
-
-Non-strict contiguity means that other events are allowed to occur in-between 
two matching events.
-A non-strict contiguity pattern state can be created via the `followedBy` 
method.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-Pattern<Event, ?> nonStrictNext = start.followedBy("middle");
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val nonStrictNext : Pattern[Event, _] = start.followedBy("middle")
-{% endhighlight %}
-</div>
-</div>
-It is also possible to define a temporal constraint for the pattern to be 
valid.
-For example, one can define that a pattern should occur within 10 seconds via 
the `within` method.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-next.within(Time.seconds(10));
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-next.within(Time.seconds(10))
-{% endhighlight %}
-</div>
-</div>
-
-<br />
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-<table class="table table-bordered">
-    <thead>
-        <tr>
-            <th class="text-left" style="width: 25%">Pattern Operation</th>
-            <th class="text-center">Description</th>
-        </tr>
-    </thead>
-    <tbody>
-        <tr>
-            <td><strong>Begin</strong></td>
-            <td>
-            <p>Defines a starting pattern state:</p>
-{% highlight java %}
-Pattern<Event, ?> start = Pattern.<Event>begin("start");
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Next</strong></td>
-            <td>
-                <p>Appends a new pattern state. A matching event has to 
directly succeed the previous matching event:</p>
-{% highlight java %}
-Pattern<Event, ?> next = start.next("next");
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>FollowedBy</strong></td>
-            <td>
-                <p>Appends a new pattern state. Other events can occur between 
a matching event and the previous matching event:</p>
-{% highlight java %}
-Pattern<Event, ?> followedBy = start.followedBy("next");
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Where</strong></td>
-            <td>
-                <p>Defines a filter condition for the current pattern state. 
Only if an event passes the filter, it can match the state:</p>
-{% highlight java %}
-patternState.where(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) throws Exception {
-        return ... // some condition
-    }
-});
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Or</strong></td>
-            <td>
-                <p>Adds a new filter condition which is ORed with an existing 
filter condition. Only if an event passes the filter condition, it can match 
the state:</p>
-{% highlight java %}
-patternState.where(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) throws Exception {
-        return ... // some condition
-    }
-}).or(new FilterFunction<Event>() {
-    @Override
-    public boolean filter(Event value) throws Exception {
-        return ... // alternative condition
-    }
-});
-{% endhighlight %}
-                    </td>
-                </tr>
-       <tr>
-           <td><strong>Subtype</strong></td>
-           <td>
-               <p>Defines a subtype condition for the current pattern state. 
Only if an event is of this subtype, it can match the state:</p>
-{% highlight java %}
-patternState.subtype(SubEvent.class);
-{% endhighlight %}
-           </td>
-       </tr>
-       <tr>
-          <td><strong>Within</strong></td>
-          <td>
-              <p>Defines the maximum time interval for an event sequence to 
match the pattern. If a non-completed event sequence exceeds this time, it is 
discarded:</p>
-{% highlight java %}
-patternState.within(Time.seconds(10));
-{% endhighlight %}
-          </td>
-      </tr>
-  </tbody>
-</table>
-</div>
-
-<div data-lang="scala" markdown="1">
-<table class="table table-bordered">
-    <thead>
-        <tr>
-            <th class="text-left" style="width: 25%">Pattern Operation</th>
-            <th class="text-center">Description</th>
-        </tr>
-    </thead>
-    <tbody>
-        <tr>
-            <td><strong>Begin</strong></td>
-            <td>
-            <p>Defines a starting pattern state:</p>
-{% highlight scala %}
-val start = Pattern.begin[Event]("start")
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Next</strong></td>
-            <td>
-                <p>Appends a new pattern state. A matching event has to 
directly succeed the previous matching event:</p>
-{% highlight scala %}
-val next = start.next("middle")
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>FollowedBy</strong></td>
-            <td>
-                <p>Appends a new pattern state. Other events can occur between 
a matching event and the previous matching event:</p>
-{% highlight scala %}
-val followedBy = start.followedBy("middle")
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Where</strong></td>
-            <td>
-                <p>Defines a filter condition for the current pattern state. 
Only if an event passes the filter, it can match the state:</p>
-{% highlight scala %}
-patternState.where(event => ... /* some condition */)
-{% endhighlight %}
-            </td>
-        </tr>
-        <tr>
-            <td><strong>Or</strong></td>
-            <td>
-                <p>Adds a new filter condition which is ORed with an existing 
filter condition. Only if an event passes the filter condition, it can match 
the state:</p>
-{% highlight scala %}
-patternState.where(event => ... /* some condition */)
-    .or(event => ... /* alternative condition */)
-{% endhighlight %}
-                    </td>
-                </tr>
-       <tr>
-           <td><strong>Subtype</strong></td>
-           <td>
-               <p>Defines a subtype condition for the current pattern state. 
Only if an event is of this subtype, it can match the state:</p>
-{% highlight scala %}
-patternState.subtype(classOf[SubEvent])
-{% endhighlight %}
-           </td>
-       </tr>
-       <tr>
-          <td><strong>Within</strong></td>
-          <td>
-              <p>Defines the maximum time interval for an event sequence to 
match the pattern. If a non-completed event sequence exceeds this time, it is 
discarded:</p>
-{% highlight scala %}
-patternState.within(Time.seconds(10))
-{% endhighlight %}
-          </td>
-      </tr>
-  </tbody>
-</table>
-</div>
-
-</div>
-
-### Detecting Patterns
-
-In order to run a stream of events against your pattern, you have to create a 
`PatternStream`.
-Given an input stream `input` and a pattern `pattern`, you create the 
`PatternStream` by calling
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-DataStream<Event> input = ...
-Pattern<Event, ?> pattern = ...
-
-PatternStream<Event> patternStream = CEP.pattern(input, pattern);
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val input : DataStream[Event] = ...
-val pattern : Pattern[Event, _] = ...
-
-val patternStream: PatternStream[Event] = CEP.pattern(input, pattern)
-{% endhighlight %}
-</div>
-</div>
-
-### Selecting from Patterns
-Once you have obtained a `PatternStream` you can select from detected event 
sequences via the `select` or `flatSelect` methods.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-The `select` method requires a `PatternSelectFunction` implementation.
-A `PatternSelectFunction` has a `select` method which is called for each 
matching event sequence.
-It receives a map of string/event pairs of the matched events.
-The string is defined by the name of the state to which the event has been 
matched.
-The `select` method can return exactly one result.
-
-{% highlight java %}
-class MyPatternSelectFunction<IN, OUT> implements PatternSelectFunction<IN, 
OUT> {
-    @Override
-    public OUT select(Map<String, IN> pattern) {
-        IN startEvent = pattern.get("start");
-        IN endEvent = pattern.get("end");
-        return new OUT(startEvent, endEvent);
-    }
-}
-{% endhighlight %}
-
-A `PatternFlatSelectFunction` is similar to the `PatternSelectFunction`, with 
the only distinction that it can return an arbitrary number of results.
-In order to do this, the `select` method has an additional `Collector` 
parameter which is used for the element output.
-
-{% highlight java %}
-class MyPatternFlatSelectFunction<IN, OUT> implements 
PatternFlatSelectFunction<IN, OUT> {
-    @Override
-    public void select(Map<String, IN> pattern, Collector<OUT> collector) {
-        IN startEvent = pattern.get("start");
-        IN endEvent = pattern.get("end");
-
-        for (int i = 0; i < startEvent.getValue(); i++ ) {
-            collector.collect(new OUT(startEvent, endEvent));
-        }
-    }
-}
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-The `select` method takes a selection function as argument, which is called 
for each matching event sequence.
-It receives a map of string/event pairs of the matched events.
-The string is defined by the name of the state to which the event has been 
matched.
-The selection function returns exactly one result per call.
-
-{% highlight scala %}
-def selectFn(pattern : mutable.Map[String, IN]): OUT = {
-    val startEvent = pattern.get("start").get
-    val endEvent = pattern.get("end").get
-    OUT(startEvent, endEvent)
-}
-{% endhighlight %}
-
-The `flatSelect` method is similar to the `select` method. Their only 
difference is that the function passed to the `flatSelect` method can return an 
arbitrary number of results per call.
-In order to do this, the function for `flatSelect` has an additional 
`Collector` parameter which is used for the element output.
-
-{% highlight scala %}
-def flatSelectFn(pattern : mutable.Map[String, IN], collector : 
Collector[OUT]) = {
-    val startEvent = pattern.get("start").get
-    val endEvent = pattern.get("end").get
-    for (i <- 0 to startEvent.getValue) {
-        collector.collect(OUT(startEvent, endEvent))
-    }
-}
-{% endhighlight %}
-</div>
-</div>
-
-### Handling Timed Out Partial Patterns
-
-Whenever a pattern has a window length associated via the `within` key word, 
it is possible that partial event patterns will be discarded because they 
exceed the window length.
-In order to react to these timeout events the `select` and `flatSelect` API 
calls allow to specify a timeout handler.
-This timeout handler is called for each partial event pattern which has timed 
out.
-The timeout handler receives all so far matched events of the partial pattern 
and the timestamp when the timeout was detected.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-In order to treat partial patterns, the `select` and `flatSelect` API calls 
offer an overloaded version which takes as the first parameter a 
`PatternTimeoutFunction`/`PatternFlatTimeoutFunction` and as second parameter 
the known `PatternSelectFunction`/`PatternFlatSelectFunction`.
-The return type of the timeout function can be different from the select 
function.
-The timeout event and the select event are wrapped in `Either.Left` and 
`Either.Right` respectively so that the resulting data stream is of type 
`org.apache.flink.types.Either`.
-
-{% highlight java %}
-PatternStream<Event> patternStream = CEP.pattern(input, pattern);
-
-DataStream<Either<TimeoutEvent, ComplexEvent>> result = patternStream.select(
-    new PatternTimeoutFunction<Event, TimeoutEvent>() {...},
-    new PatternSelectFunction<Event, ComplexEvent>() {...}
-);
-
-DataStream<Either<TimeoutEvent, ComplexEvent>> flatResult = 
patternStream.flatSelect(
-    new PatternFlatTimeoutFunction<Event, TimeoutEvent>() {...},
-    new PatternFlatSelectFunction<Event, ComplexEvent>() {...}
-);
-{% endhighlight %}
-
-</div>
-
-<div data-lang="scala" markdown="1">
-In order to treat partial patterns, the `select` API call offers an overloaded 
version which takes as the first parameter a timeout function and as second 
parameter a selection function.
-The timeout function is called with a map of string-event pairs of the partial 
match which has timed out and a long indicating when the timeout occurred.
-The string is defined by the name of the state to which the event has been 
matched.
-The timeout function returns exactly one result per call.
-The return type of the timeout function can be different from the select 
function.
-The timeout event and the select event are wrapped in `Left` and `Right` 
respectively so that the resulting data stream is of type `Either`.
-
-{% highlight scala %}
-val patternStream: PatternStream[Event] = CEP.pattern(input, pattern)
-
-DataStream[Either[TimeoutEvent, ComplexEvent]] result = patternStream.select{
-    (pattern: mutable.Map[String, Event], timestamp: Long) => TimeoutEvent()
-} {
-    pattern: mutable.Map[String, Event] => ComplexEvent()
-}
-{% endhighlight %}
-
-The `flatSelect` API call offers the same overloaded version which takes as 
the first parameter a timeout function and as second parameter a selection 
function.
-In contrast to the `select` functions, the `flatSelect` functions are called 
with an `Collector`.
-The collector can be used to emit an arbitrary number of events.
-
-{% highlight scala %}
-val patternStream: PatternStream[Event] = CEP.pattern(input, pattern)
-
-DataStream[Either[TimeoutEvent, ComplexEvent]] result = 
patternStream.flatSelect{
-    (pattern: mutable.Map[String, Event], timestamp: Long, out: 
Collector[TimeoutEvent]) =>
-        out.collect(TimeoutEvent())
-} {
-    (pattern: mutable.Map[String, Event], out: Collector[ComplexEvent]) =>
-        out.collect(ComplexEvent())
-}
-{% endhighlight %}
-
-</div>
-</div>
-
-## Examples
-
-The following example detects the pattern `start, middle(name = "error") -> 
end(name = "critical")` on a keyed data stream of `Events`.
-The events are keyed by their ids and a valid pattern has to occur within 10 
seconds.
-The whole processing is done with event time.
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-{% highlight java %}
-StreamExecutionEnvironment env = ...
-env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-DataStream<Event> input = ...
-
-DataStream<Event> partitionedInput = input.keyBy(new KeySelector<Event, 
Integer>() {
-       @Override
-       public Integer getKey(Event value) throws Exception {
-               return value.getId();
-       }
-});
-
-Pattern<Event, ?> pattern = Pattern.<Event>begin("start")
-       .next("middle").where(new FilterFunction<Event>() {
-               @Override
-               public boolean filter(Event value) throws Exception {
-                       return value.getName().equals("error");
-               }
-       }).followedBy("end").where(new FilterFunction<Event>() {
-               @Override
-               public boolean filter(Event value) throws Exception {
-                       return value.getName().equals("critical");
-               }
-       }).within(Time.seconds(10));
-
-PatternStream<Event> patternStream = CEP.pattern(partitionedInput, pattern);
-
-DataStream<Alert> alerts = patternStream.select(new 
PatternSelectFunction<Event, Alert>() {
-       @Override
-       public Alert select(Map<String, Event> pattern) throws Exception {
-               return createAlert(pattern);
-       }
-});
-{% endhighlight %}
-</div>
-
-<div data-lang="scala" markdown="1">
-{% highlight scala %}
-val env : StreamExecutionEnvironment = ...
-env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-
-val input : DataStream[Event] = ...
-
-val partitionedInput = input.keyBy(event => event.getId)
-
-val pattern = Pattern.begin("start")
-  .next("middle").where(_.getName == "error")
-  .followedBy("end").where(_.getName == "critical")
-  .within(Time.seconds(10))
-
-val patternStream = CEP.pattern(partitionedInput, pattern)
-
-val alerts = patternStream.select(createAlert(_)))
-{% endhighlight %}
-</div>
-</div>

http://git-wip-us.apache.org/repos/asf/flink/blob/844c874b/docs/apis/streaming/libs/index.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming/libs/index.md 
b/docs/apis/streaming/libs/index.md
deleted file mode 100644
index 4ba7e94..0000000
--- a/docs/apis/streaming/libs/index.md
+++ /dev/null
@@ -1,27 +0,0 @@
----
-title: "Streaming Libraries"
-sub-nav-group: streaming
-sub-nav-id: libs
-sub-nav-pos: 7
-sub-nav-title: Libraries
----
-<!--
-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.
--->
-
-- Complex event processing: [CEP](cep.html)

http://git-wip-us.apache.org/repos/asf/flink/blob/844c874b/docs/apis/streaming/non-windowed.svg
----------------------------------------------------------------------
diff --git a/docs/apis/streaming/non-windowed.svg 
b/docs/apis/streaming/non-windowed.svg
deleted file mode 100644
index 3c1cdaa..0000000
--- a/docs/apis/streaming/non-windowed.svg
+++ /dev/null
@@ -1,22 +0,0 @@
-<?xml version="1.0" standalone="yes"?>
-<!--
-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.
--->
-
-<svg version="1.1" viewBox="0.0 0.0 800.0 600.0" fill="none" stroke="none" 
stroke-linecap="square" stroke-miterlimit="10" 
xmlns="http://www.w3.org/2000/svg"; 
xmlns:xlink="http://www.w3.org/1999/xlink";><clipPath id="p.0"><path d="m0 
0l800.0 0l0 600.0l-800.0 0l0 -600.0z" clip-rule="nonzero"></path></clipPath><g 
clip-path="url(#p.0)"><path fill="#000000" fill-opacity="0.0" d="m0 0l800.0 0l0 
600.0l-800.0 0z" fill-rule="nonzero"></path><path fill="#000000" 
fill-opacity="0.0" d="m145.49606 485.0l509.0079 0" 
fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" 
stroke-linejoin="round" stroke-linecap="butt" d="m145.49606 485.0l503.0079 0" 
fill-rule="evenodd"></path><path fill="#000000" stroke="#000000" 
stroke-width="1.0" stroke-linecap="butt" d="m648.50397 486.65173l4.538086 
-1.6517334l-4.538086 -1.6517334z" fill-rule="evenodd"></path><path 
fill="#000000" fill-opacity="0.0" d="m145.49606 485.0l0 -394.99213" 
fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" s
 troke-linejoin="round" stroke-linecap="butt" d="m145.49606 485.0l0 -388.99213" 
fill-rule="evenodd"></path><path fill="#000000" stroke="#000000" 
stroke-width="1.0" stroke-linecap="butt" d="m147.1478 96.00787l-1.6517334 
-4.5380936l-1.6517334 4.5380936z" fill-rule="evenodd"></path><path 
fill="#000000" fill-opacity="0.0" d="m587.0 477.0l60.0 0l0 42.992126l-60.0 0z" 
fill-rule="nonzero"></path><path fill="#000000" d="m600.90625 
502.41998l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 
-1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 
-1.984375l0 -5.65625l-1.234375 0l0 -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 
3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 
0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 
-0.078125zm1.5426636 -10.1875l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 
11.6875l0 -9.859375l1.671875 0l0 9.859375l-1.671875 0zm4.1292114 0l0 
-9.859375l1.5 0l0 1.390625q0.453125 
 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 
1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 
3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 
6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 
-0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 
0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 
-1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 
-1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 
2.0625l0 5.109375l-1.671875 0zm22.290771 -3.171875l1.71875 0.21875q-0.40625 1.5 
-1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 
-1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 
-3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 
1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 
0.921875 2.484375q0.828125 0.85
 9375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 
-1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 
-1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 
0.78125q-0.78125 0.765625 -0.859375 2.046875z" fill-rule="nonzero"></path><path 
fill="#000000" fill-opacity="0.0" d="m42.0 133.0l82.01575 0l0 
42.992126l-82.01575 0z" fill-rule="nonzero"></path><path fill="#000000" 
d="m58.703125 159.92l0 -1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 
-1.625 -0.328125q-0.75 -0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 
-1.234375q-0.09375 -0.5 -0.09375 -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 
1.3125 0.09375 1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 
1.265625 0.375q0.75 0 1.40625 -0.375q0.65625 -0.390625 0.921875 
-1.046875q0.28125 -0.671875 0.28125 -1.9375l0 -5.28125l1.671875 0l0 
9.859375l-1.5 0zm3.2507172 -2.9375l1.65625 -0.265625q0.140625 1.0 0.765625 
1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 -0.46
 875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 
-0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 
-0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 
0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 
-0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 
0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 
0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 
-0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 
0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 
1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 
0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 
0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 
-2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm16.75 -0.234375l1.71875 
0.21875q-0.40625 1.5 -1.515625 2.
 34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 
-1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 
-3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 
1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 
0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 
-0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 
0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 
-0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 
2.046875zm9.094467 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 
-1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 
1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 
0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 
1.953125l0 5.15625l-1.671875 0zm17.23973 0l-1.671875 0l0 -10.640625q-0.59375 
0.578125 -1.578125 1.15625q-0.984375 0.5625 -1.765625 0.859375l0
  -1.625q1.40625 -0.65625 2.453125 -1.59375q1.046875 -0.9375 1.484375 
-1.8125l1.078125 0l0 13.65625z" fill-rule="nonzero"></path><path fill="#000000" 
fill-opacity="0.0" d="m42.0 254.0l82.01575 0l0 42.992126l-82.01575 0z" 
fill-rule="nonzero"></path><path fill="#000000" d="m58.703125 280.91998l0 
-1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 -1.625 -0.328125q-0.75 
-0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 -1.234375q-0.09375 -0.5 
-0.09375 -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 1.3125 0.09375 
1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 1.265625 0.375q0.75 0 
1.40625 -0.375q0.65625 -0.390625 0.921875 -1.046875q0.28125 -0.671875 0.28125 
-1.9375l0 -5.28125l1.671875 0l0 9.859375l-1.5 0zm3.2507172 -2.9375l1.65625 
-0.265625q0.140625 1.0 0.765625 1.53125q0.640625 0.515625 1.78125 
0.515625q1.15625 0 1.703125 -0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 
-0.484375 -0.890625q-0.34375 -0.21875 -1.703125 -0.5625q-1.84375 -0.46875 
-2.5625 -0.796875q-0.703125 -0.
 34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 
0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 
-0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 
0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 
0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 
-0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 
0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 
1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 
0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 
0.734375 -1.40625 1.140625q-0.921875 0.390625 -2.078125 0.390625q-1.921875 0 
-2.9375 -0.796875q-1.0 -0.796875 -1.28125 -2.359375zm16.75 -0.234375l1.71875 
0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 
0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 
-3.734375q0 -2.484375 1.265625 -3.859375q1.28125 
 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 
3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 
2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 
-0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 
-0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 
0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.094467 5.875l0 -9.859375l1.5 
0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 
-0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 
-1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 
0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0zm19.724106 
-1.609375l0 1.609375l-8.984375 0q-0.015625 -0.609375 0.1875 -1.15625q0.34375 
-0.921875 1.09375 -1.8125q0.765625 -0.890625 2.1875 -2.0625q2.21875 -1.8125 3.0 
-2.875q0.78125 -1.0625 0.78125 -2.015625q0 -0.984375 -0.71875 
-1.671875q-0.703125 -0.6875 -1.
 84375 -0.6875q-1.203125 0 -1.9375 0.734375q-0.71875 0.71875 -0.71875 
2.0l-1.71875 -0.171875q0.171875 -1.921875 1.328125 -2.921875q1.15625 -1.015625 
3.09375 -1.015625q1.953125 0 3.09375 1.09375q1.140625 1.078125 1.140625 
2.6875q0 0.8125 -0.34375 1.609375q-0.328125 0.78125 -1.109375 1.65625q-0.765625 
0.859375 -2.5625 2.390625q-1.5 1.265625 -1.9375 1.71875q-0.421875 0.4375 
-0.703125 0.890625l6.671875 0z" fill-rule="nonzero"></path><path fill="#000000" 
fill-opacity="0.0" d="m42.0 375.0l82.01575 0l0 42.992126l-82.01575 0z" 
fill-rule="nonzero"></path><path fill="#000000" d="m58.703125 401.91998l0 
-1.453125q-1.140625 1.671875 -3.125 1.671875q-0.859375 0 -1.625 -0.328125q-0.75 
-0.34375 -1.125 -0.84375q-0.359375 -0.5 -0.515625 -1.234375q-0.09375 -0.5 
-0.09375 -1.5625l0 -6.109375l1.671875 0l0 5.46875q0 1.3125 0.09375 
1.765625q0.15625 0.65625 0.671875 1.03125q0.515625 0.375 1.265625 0.375q0.75 0 
1.40625 -0.375q0.65625 -0.390625 0.921875 -1.046875q0.28125 -0.671875 0.28125 
-1.9375l0 -5.28125l1.
 671875 0l0 9.859375l-1.5 0zm3.2507172 -2.9375l1.65625 -0.265625q0.140625 1.0 
0.765625 1.53125q0.640625 0.515625 1.78125 0.515625q1.15625 0 1.703125 
-0.46875q0.5625 -0.46875 0.5625 -1.09375q0 -0.5625 -0.484375 -0.890625q-0.34375 
-0.21875 -1.703125 -0.5625q-1.84375 -0.46875 -2.5625 -0.796875q-0.703125 
-0.34375 -1.078125 -0.9375q-0.359375 -0.609375 -0.359375 -1.328125q0 -0.65625 
0.296875 -1.21875q0.3125 -0.5625 0.828125 -0.9375q0.390625 -0.28125 1.0625 
-0.484375q0.671875 -0.203125 1.4375 -0.203125q1.171875 0 2.046875 0.34375q0.875 
0.328125 1.28125 0.90625q0.421875 0.5625 0.578125 1.515625l-1.625 
0.21875q-0.109375 -0.75 -0.65625 -1.171875q-0.53125 -0.4375 -1.5 
-0.4375q-1.15625 0 -1.640625 0.390625q-0.484375 0.375 -0.484375 0.875q0 
0.328125 0.203125 0.59375q0.203125 0.265625 0.640625 0.4375q0.25 0.09375 
1.46875 0.4375q1.765625 0.46875 2.46875 0.765625q0.703125 0.296875 1.09375 
0.875q0.40625 0.578125 0.40625 1.4375q0 0.828125 -0.484375 1.578125q-0.484375 
0.734375 -1.40625 1.140625q-0.9218
 75 0.390625 -2.078125 0.390625q-1.921875 0 -2.9375 -0.796875q-1.0 -0.796875 
-1.28125 -2.359375zm16.75 -0.234375l1.71875 0.21875q-0.40625 1.5 -1.515625 
2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 
-1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 
-3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 
1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 
0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 
-0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 
0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 
-0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 
2.046875zm9.094467 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 
-1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 
1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 
0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.8906
 25 -0.28125 1.953125l0 5.15625l-1.671875 0zm10.958481 -3.59375l1.671875 
-0.21875q0.28125 1.421875 0.96875 2.046875q0.703125 0.625 1.6875 0.625q1.1875 0 
2.0 -0.8125q0.8125 -0.828125 0.8125 -2.03125q0 -1.140625 -0.765625 
-1.890625q-0.75 -0.75 -1.90625 -0.75q-0.46875 0 -1.171875 0.1875l0.1875 
-1.46875q0.15625 0.015625 0.265625 0.015625q1.0625 0 1.90625 -0.546875q0.859375 
-0.5625 0.859375 -1.71875q0 -0.921875 -0.625 -1.515625q-0.609375 -0.609375 
-1.59375 -0.609375q-0.96875 0 -1.625 0.609375q-0.640625 0.609375 -0.828125 
1.84375l-1.671875 -0.296875q0.296875 -1.6875 1.375 -2.609375q1.09375 -0.921875 
2.71875 -0.921875q1.109375 0 2.046875 0.484375q0.9375 0.46875 1.421875 
1.296875q0.5 0.828125 0.5 1.75q0 0.890625 -0.46875 1.609375q-0.46875 0.71875 
-1.40625 1.15625q1.21875 0.265625 1.875 1.15625q0.671875 0.875 0.671875 
2.1875q0 1.78125 -1.296875 3.015625q-1.296875 1.234375 -3.28125 
1.234375q-1.796875 0 -2.984375 -1.0625q-1.171875 -1.0625 -1.34375 -2.765625z" 
fill-rule="nonzero"></path><path fi
 ll="#9900ff" d="m177.0 154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 
-9.496063l0 0c2.518509 0 4.9338684 1.000473 6.714737 2.7813263c1.7808533 
1.7808685 2.7813263 4.196228 2.7813263 6.714737l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m203.49606 
154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.518509 0 
4.9338684 1.000473 6.714737 2.7813263c1.7808533 1.7808685 2.7813263 4.196228 
2.7813263 6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 
0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m290.0 154.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.000473 6.7147217 2.7813263c1.7808533 1.7808685 2.7813416 4.196228 2.7813416 
6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 0 
-9.496063 -4.251526 -9.4960
 63 -9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m323.0 
154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 
0 4.9338684 1.000473 6.7147217 2.7813263c1.7808533 1.7808685 2.7813416 4.196228 
2.7813416 6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 
0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m348.0 154.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.000473 6.7147217 2.7813263c1.7808533 1.7808685 2.7813416 4.196228 2.7813416 
6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 0 
-9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m373.0 154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 
-9.496063l0 0c2.5185242 0 4.9338684 1.000473 6.7147217 2.7813263c1.7808533 
1.7808685 2.7813416 4.196228 2.7813416 6.714737l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.4
 96063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m442.50394 154.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.000473 6.7147217 2.7813263c1.7808533 1.7808685 2.7813416 4.196228 2.7813416 
6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 0 
-9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m469.0 154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 
-9.496063l0 0c2.5185242 0 4.9338684 1.000473 6.7147217 2.7813263c1.7808533 
1.7808685 2.7813416 4.196228 2.7813416 6.714737l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m492.50394 
154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 
0 4.9338684 1.000473 6.7147217 2.7813263c1.7808533 1.7808685 2.7813416 4.196228 
2.7813416
  6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 
0 -9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m524.0 154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496033 
-9.496063l0 0c2.5185547 0 4.933899 1.000473 6.7147827 2.7813263c1.7808228 
1.7808685 2.781311 4.196228 2.781311 6.714737l0 0c0 5.2445374 -4.251526 
9.496063 -9.496094 9.496063l0 0c-5.244507 0 -9.496033 -4.251526 -9.496033 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m603.0079 
154.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496094 -9.496063l0 0c2.5184937 
0 4.933838 1.000473 6.7147217 2.7813263c1.7808228 1.7808685 2.781311 4.196228 
2.781311 6.714737l0 0c0 5.2445374 -4.251526 9.496063 -9.496033 9.496063l0 
0c-5.244568 0 -9.496094 -4.251526 -9.496094 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m374.97638 275.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.0004883 6.7147217 2.781341
 6c1.7808533 1.7808533 2.7813416 4.1961975 2.7813416 6.7147217l0 0c0 5.2445374 
-4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 
-9.496063 -9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" 
d="m401.47244 275.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 
-9.496063l0 0c2.5185242 0 4.9338684 1.0004883 6.7147217 2.7813416c1.7808533 
1.7808533 2.7813416 4.1961975 2.7813416 6.7147217l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m209.0 
275.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.518509 0 
4.9338684 1.0004883 6.714737 2.7813416c1.7808533 1.7808533 2.7813263 4.1961975 
2.7813263 6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 
0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m242.0 275.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.
 496063l0 0c2.518509 0 4.9338684 1.0004883 6.7147217 2.7813416c1.7808533 
1.7808533 2.7813416 4.1961975 2.7813416 6.7147217l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m267.0 
275.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 
0 4.9338684 1.0004883 6.7147217 2.7813416c1.7808533 1.7808533 2.7813416 
4.1961975 2.7813416 6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 
9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m292.0 275.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.0004883 6.7147217 2.7813416c1.7808533 1.7808533 2.7813416 4.1961975 2.7813416 
6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 
0 -9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m568.48
 03 275.49606l0 0c0 -5.2445374 4.251587 -9.496063 9.496094 -9.496063l0 
0c2.5184937 0 4.933899 1.0004883 6.7147217 2.7813416c1.7808838 1.7808533 
2.781311 4.1961975 2.781311 6.7147217l0 0c0 5.2445374 -4.251526 9.496063 
-9.496033 9.496063l0 0c-5.244507 0 -9.496094 -4.251526 -9.496094 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m594.9764 275.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496033 -9.496063l0 0c2.5185547 0 4.933899 
1.0004883 6.7147827 2.7813416c1.7808228 1.7808533 2.781311 4.1961975 2.781311 
6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496094 9.496063l0 0c-5.244507 0 
-9.496033 -4.251526 -9.496033 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m618.4803 275.49606l0 0c0 -5.2445374 4.251587 -9.496063 
9.496094 -9.496063l0 0c2.5184937 0 4.933899 1.0004883 6.7147217 
2.7813416c1.7808838 1.7808533 2.781311 4.1961975 2.781311 6.7147217l0 0c0 
5.2445374 -4.251526 9.496063 -9.496033 9.496063l0 0c-5.244507 0 -9.496094 
-4.251526 -9.496094 -9.496063z" fi
 ll-rule="nonzero"></path><path fill="#9900ff" d="m477.0 275.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.0004883 6.7147217 2.7813416c1.7808533 1.7808533 2.7813416 4.1961975 2.7813416 
6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 
0 -9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m487.99213 396.49606l0 0c0 -5.2445374 4.251526 -9.496063 
9.496063 -9.496063l0 0c2.5185242 0 4.9338684 1.0004883 6.7147217 
2.7813416c1.7808533 1.7808533 2.7813416 4.1961975 2.7813416 6.7147217l0 0c0 
5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 
-4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" 
d="m514.48816 396.49606l0 0c0 -5.2445374 4.251587 -9.496063 9.496094 
-9.496063l0 0c2.5184937 0 4.933899 1.0004883 6.7147217 2.7813416c1.7808838 
1.7808533 2.781311 4.1961975 2.781311 6.7147217l0 0c0 5.2445374 -4.251526 
9.496063 -9.496033 9.4960
 63l0 0c-5.244507 0 -9.496094 -4.251526 -9.496094 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m185.76378 396.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.518509 0 4.9338684 
1.0004883 6.714737 2.7813416c1.7808533 1.7808533 2.7813263 4.1961975 2.7813263 
6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 
0 -9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m265.0 396.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 
-9.496063l0 0c2.5185242 0 4.9338684 1.0004883 6.7147217 2.7813416c1.7808533 
1.7808533 2.7813416 4.1961975 2.7813416 6.7147217l0 0c0 5.2445374 -4.251526 
9.496063 -9.496063 9.496063l0 0c-5.2445374 0 -9.496063 -4.251526 -9.496063 
-9.496063z" fill-rule="nonzero"></path><path fill="#9900ff" d="m291.49606 
396.49606l0 0c0 -5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 
0 4.9338684 1.0004883 6.7147217 2.7813416c1.7808533 1.7808533 2.7813416 
4.1961975 2.78134
 16 6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 
0c-5.2445374 0 -9.496063 -4.251526 -9.496063 -9.496063z" 
fill-rule="nonzero"></path><path fill="#9900ff" d="m315.0 396.49606l0 0c0 
-5.2445374 4.251526 -9.496063 9.496063 -9.496063l0 0c2.5185242 0 4.9338684 
1.0004883 6.7147217 2.7813416c1.7808533 1.7808533 2.7813416 4.1961975 2.7813416 
6.7147217l0 0c0 5.2445374 -4.251526 9.496063 -9.496063 9.496063l0 0c-5.2445374 
0 -9.496063 -4.251526 -9.496063 -9.496063z" fill-rule="nonzero"></path><path 
fill="#9900ff" d="m558.01575 396.49606l0 0c0 -5.2445374 4.251526 -9.496063 
9.496094 -9.496063l0 0c2.5184937 0 4.933838 1.0004883 6.7147217 
2.7813416c1.7808228 1.7808533 2.781311 4.1961975 2.781311 6.7147217l0 0c0 
5.2445374 -4.251526 9.496063 -9.496033 9.496063l0 0c-5.244568 0 -9.496094 
-4.251526 -9.496094 -9.496063z" fill-rule="nonzero"></path></g></svg>
-

http://git-wip-us.apache.org/repos/asf/flink/blob/844c874b/docs/apis/streaming/savepoints.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming/savepoints.md 
b/docs/apis/streaming/savepoints.md
deleted file mode 100644
index 53a1ee8..0000000
--- a/docs/apis/streaming/savepoints.md
+++ /dev/null
@@ -1,110 +0,0 @@
----
-title: "Savepoints"
-is_beta: false
-sub-nav-group: streaming
-sub-nav-pos: 6
----
-<!--
-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](index.html) can resume execution 
from a **savepoint**. Savepoints allow both updating your programs and your 
Flink cluster without losing any state. This page covers all steps to trigger, 
restore, and dispose savepoints. For more details on how Flink handles state 
and failures, check out the [State in Streaming Programs](state_backends.html) 
and [Fault Tolerance](fault_tolerance.html) pages.
-
-* toc
-{:toc}
-
-## Overview
-
-Savepoints are **manually triggered checkpoints**, which take a snapshot of 
the program and write it out to a state backend. They rely on the regular 
checkpointing mechanism for this. During execution programs are periodically 
snapshotted on the worker nodes and produce checkpoints. For recovery only the 
last completed checkpoint is needed and older checkpoints can be safely 
discarded as soon as a new one is completed.
-
-Savepoints are similar to these periodic checkpoints except that they are 
**triggered by the user** and **don't automatically expire** when newer 
checkpoints are completed.
-
-<img src="fig/savepoints-overview.png" class="center" />
-
-In the above example the workers produce checkpoints **c<sub>1</sub>**, 
**c<sub>2</sub>**, **c<sub>3</sub>**, and **c<sub>4</sub>** for job *0xA312Bc*. 
Periodic checkpoints **c<sub>1</sub>** and **c<sub>3</sub>** have already been 
*discarded* and **c<sub>4</sub>** is the *latest checkpoint*. **c<sub>2</sub> 
is special**. It is the state associated with the savepoint **s<sub>1</sub>** 
and has been triggered by the user and it doesn't expire automatically (as 
c<sub>1</sub> and c<sub>3</sub> did after the completion of newer checkpoints).
-
-Note that **s<sub>1</sub>** is only a **pointer to the actual checkpoint data 
c<sub>2</sub>**. This means that the actual state is *not copied* for the 
savepoint and periodic checkpoint data is kept around.
-
-## Configuration
-
-Savepoints point to regular checkpoints and store their state in a configured 
[state backend](state_backends.html). Currently, the supported state backends 
are **jobmanager** and **filesystem**. The state backend configuration for the 
regular periodic checkpoints is **independent** of the savepoint state backend 
configuration. Checkpoint data is **not copied** for savepoints, but points to 
the configured checkpoint state backend.
-
-### JobManager
-
-This is the **default backend** for savepoints.
-
-Savepoints are stored on the heap of the job manager. They are *lost* after 
the job manager is shut down. This mode is only useful if you want to *stop* 
and *resume* your program while the **same cluster** keeps running. It is *not 
recommended* for production use. Savepoints are *not* part of the [job 
manager's highly available]({{ site.baseurl 
}}/setup/jobmanager_high_availability.html) state.
-
-<pre>
-savepoints.state.backend: jobmanager
-</pre>
-
-**Note**: If you don't configure a specific state backend for the savepoints, 
the jobmanager backend will be used.
-
-### File system
-
-Savepoints are stored in the configured **file system directory**. They are 
available between cluster instances and allow to move your program to another 
cluster.
-
-<pre>
-savepoints.state.backend: filesystem
-savepoints.state.backend.fs.dir: hdfs:///flink/savepoints
-</pre>
-
-**Note**: If you don't configure a specific directory, the job manager backend 
will be used.
-
-**Important**: A savepoint is a pointer to a completed checkpoint. That means 
that the state of a savepoint is not only found in the savepoint file itself, 
but also needs the actual checkpoint data (e.g. in a set of further files). 
Therefore, using the *filesystem* backend for savepoints and the *jobmanager* 
backend for checkpoints does not work, because the required checkpoint data 
won't be available after a job manager restart.
-
-## Changes to your program
-
-Savepoints **work out of the box**, but it is **highly recommended** that you 
slightly adjust your programs in order to be able to work with savepoints in 
future versions of your program.
-
-<img src="fig/savepoints-program_ids.png" class="center" />
-
-For savepoints **only stateful tasks matter**. In the above example, the 
source and map tasks are stateful whereas the sink is not stateful. Therefore, 
only the state of the source and map tasks are part of the savepoint.
-
-Each task is identified by its **generated task IDs** and **subtask index**. 
In the above example the state of the source (**s<sub>1</sub>**, 
**s<sub>2</sub>**) and map tasks (**m<sub>1</sub>**, **m<sub>2</sub>**) is 
identified by their respective task ID (*0xC322EC* for the source tasks and 
*0x27B3EF* for the map tasks) and subtask index. There is no state for the 
sinks (**t<sub>1</sub>**, **t<sub>2</sub>**). Their IDs therefore do not matter.
-
-<span class="label label-danger">Important</span> The IDs are generated 
**deterministically** from your program structure. This means that as long as 
your program does not change, the IDs do not change. **The only allowed changes 
are within the user function, e.g. you can change the implemented `MapFunction` 
without changing the topology**. In this case, it is straight forward to 
restore the state from a savepoint by mapping it back to the same task IDs and 
subtask indexes. This allows you to work with savepoints out of the box, but 
gets problematic as soon as you make changes to the topology, because they 
result in changed IDs and the savepoint state cannot be mapped to your program 
any more.
-
-<span class="label label-info">Recommended</span> In order to be able to 
change your program and **have fixed IDs**, the *DataStream* API provides a 
method to manually specify the task IDs. Each operator provides a 
**`uid(String)`** method to override the generated ID. The ID is a String, 
which will be deterministically hashed to a 16-byte hash value. It is 
**important** that the specified IDs are **unique per transformation and job**. 
If this is not the case, job submission will fail.
-
-{% highlight scala %}
-DataStream<String> stream = env.
-  // Stateful source (e.g. Kafka) with ID
-  .addSource(new StatefulSource())
-  .uid("source-id")
-  .shuffle()
-  // The stateful mapper with ID
-  .map(new StatefulMapper())
-  .uid("mapper-id")
-
-// Stateless sink (no specific ID required)
-stream.print()
-{% endhighlight %}
-
-## Command-line client
-
-You control the savepoints via the [command line 
client]({{site.baseurl}}/apis/cli.html#savepoints).
-
-## Current limitations
-
-- **Parallelism**: When restoring a savepoint, the parallelism of the program 
has to match the parallelism of the original program from which the savepoint 
was drawn. There is no mechanism to re-partition the savepoint's state yet.
-
-- **Chaining**: Chained operators are identified by the ID of the first task. 
It's not possible to manually assign an ID to an intermediate chained task, 
e.g. in the chain `[  a -> b -> c ]` only **a** can have its ID assigned 
manually, but not **b** or **c**. To work around this, you can [manually define 
the task chains](index.html#task-chaining-and-resource-groups). If you rely on 
the automatic ID assignment, a change in the chaining behaviour will also 
change the IDs.
-
-- **Disposing custom state handles**: Disposing an old savepoint does not work 
with custom state handles (if you are using a custom state backend), because 
the user code class loader is not available during disposal.

Reply via email to