You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by uc...@apache.org on 2014/06/23 14:52:20 UTC

[1/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Repository: incubator-flink
Updated Branches:
  refs/heads/master b4b633eab -> 40b94f733


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/spargel_guide.md
----------------------------------------------------------------------
diff --git a/docs/spargel_guide.md b/docs/spargel_guide.md
new file mode 100644
index 0000000..5766f8b
--- /dev/null
+++ b/docs/spargel_guide.md
@@ -0,0 +1,112 @@
+---
+title: "Spargel Graph Processing API"
+---
+
+Spargel
+=======
+
+Spargel is our [Giraph](http://giraph.apache.org) like **graph processing** Java API. It supports basic graph computations, which are run as a sequence of [supersteps]({{ site.baseurl }}/docs/0.4/programming_guides/iterations.html#supersteps). Spargel and Giraph both implement the [Bulk Synchronous Parallel (BSP)](https://en.wikipedia.org/wiki/Bulk_Synchronous_Parallel) programming model, propsed by Google's [Pregel](http://googleresearch.blogspot.de/2009/06/large-scale-graph-computing-at-google.html).
+
+The API provides a **vertex-centric** view on graph processing with two basic operations per superstep:
+
+  1. **Send messages** to other vertices, and
+  2. **Receive messages** from other vertices and **update own vertex state**.
+
+This vertex-centric view makes it easy to express a large class of graph problems efficiently. We will list all *relevant interfaces* of the **Spargel API** to implement and walk through an **example Spargel program**.
+
+Spargel API
+-----------
+
+The Spargel API is part of the *addons* Maven project. All relevant classes are located in the *eu.stratosphere.spargel.java* package.
+
+Add the following dependency to your `pom.xml` to use the Spargel.
+
+```xml
+<dependency>
+	<groupId>eu.stratosphere</groupId>
+	<artifactId>spargel</artifactId>
+	<version>{{site.current_stable}}</version>
+</dependency>
+```
+
+Extend **VertexUpdateFunction&lt;***VertexKeyType*, *VertexValueType*, *MessageType***&gt;** to implement your *custom vertex update logic*.
+
+Extend **MessagingFunction&lt;***VertexKeyType*, *VertexValueType*, *MessageType*, *EdgeValueType***&gt;** to implement your *custom message logic*.
+
+Create a **SpargelIteration** operator to include Spargel in your data flow.
+
+Example: Propagate Minimum Vertex ID in Graph
+---------------------------------------------
+
+The Spargel operator **SpargelIteration** includes Spargel graph processing into your data flow. As usual, it can be combined with other operators like *map*, *reduce*, *join*, etc.
+
+{% highlight java %}
+FileDataSource vertices = new FileDataSource(...);
+FileDataSource edges = new FileDataSource(...);
+
+SpargelIteration iteration = new SpargelIteration(new MinMessager(), new MinNeighborUpdater());
+iteration.setVertexInput(vertices);
+iteration.setEdgesInput(edges);
+iteration.setNumberOfIterations(maxIterations);
+
+FileDataSink result = new FileDataSink(...);
+result.setInput(iteration.getOutput());
+
+new Plan(result);
+{% endhighlight %}
+Besides the **program logic** of vertex updates in *MinNeighborUpdater* and messages in *MinMessager*, you have to specify the **initial vertex** and **edge input**. Every vertex has a **key** and **value**. In each superstep, it **receives messages** from other vertices and updates its state:
+
+  - **Vertex** input: **(id**: *VertexKeyType*, **value**: *VertexValueType***)**
+  - **Edge** input: **(source**: *VertexKeyType*, **target**: *VertexKeyType*[, **value**: *EdgeValueType*])
+
+For our example, we set the vertex ID as both *id and value* (initial minimum) and *leave out the edge values* as we don't need them:
+
+<p class="text-center">
+    <img alt="Spargel Example Input" width="75%" src="{{ site.baseurl }}/docs/0.4/img/spargel_example_input.png" />
+</p>
+
+In order to **propagate the minimum vertex ID**, we iterate over all received messages (which contain the neighboring IDs) and update our value, if we found a new minimum:
+
+{% highlight java %}
+public class MinNeighborUpdater extends VertexUpdateFunction<IntValue, IntValue, IntValue> {
+	
+	@Override
+	public void updateVertex(IntValue id, IntValue currentMin, Iterator<IntValue> messages) {
+		int min = Integer.MAX_VALUE;
+
+		// iterate over all received messages
+		while (messages.hasNext()) {
+			int next = messages.next().getValue();
+			min = next < min ? next : min;
+		}
+
+		// update vertex value, if new minimum
+		if (min < currentMin.getValue()) {
+			setNewVertexValue(new IntValue(min));
+		}
+	}
+}
+{% endhighlight %}
+
+The **messages in each superstep** consist of the **current minimum ID** seen by the vertex:
+
+{% highlight java %}
+public class MinMessager extends MessagingFunction<IntValue, IntValue, IntValue, NullValue> {
+	
+	@Override
+	public void sendMessages(IntValue id, IntValue currentMin) {
+		// send current minimum to neighbors
+		sendMessageToAllNeighbors(currentMin);
+    }
+}
+{% endhighlight %}
+
+The **API-provided method** `sendMessageToAllNeighbors(MessageType)` sends the message to all neighboring vertices. It is also possible to address specific vertices with `sendMessageTo(VertexKeyType, MessageType)`.
+
+If the value of a vertex does not change during a superstep, it will **not send** any messages in the superstep. This allows to do incremental updates to the **hot (changing) parts** of the graph, while leaving **cold (steady) parts** untouched.
+
+The computation **terminates** after a specified *maximum number of supersteps* **-OR-** the *vertex states stop changing*.
+
+<p class="text-center">
+    <img alt="Spargel Example" width="75%" src="{{ site.baseurl }}/docs/0.4/img/spargel_example.png" />
+</p>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/web_client.md
----------------------------------------------------------------------
diff --git a/docs/web_client.md b/docs/web_client.md
new file mode 100644
index 0000000..98cfd69
--- /dev/null
+++ b/docs/web_client.md
@@ -0,0 +1,53 @@
+---
+title:  "Web Client"
+---
+
+Stratosphere provides a web interface to upload jobs, inspect their execution plans, and execute them. The interface is a great tool to showcase programs, debug execution plans, or demonstrate the system as a whole.
+
+# Start, Stop, and Configure the Web Interface
+
+Start the web interface by executing:
+
+    ./bin/start-webclient.sh
+
+and stop it by calling:
+
+    ./bin/stop-webclient.sh
+
+The web interface runs on port 8080 by default. To specify a custom port set the ```webclient.port``` property in the *./conf/stratosphere.yaml* configuration file. Jobs are submitted to the JobManager specified by ```jobmanager.rpc.address``` and ```jobmanager.rpc.port```. Please consult the [configuration](../setup/config.html#web_frontend "Configuration") page for details and further configuration options.
+
+# Use the Web Interface
+
+The web interface provides two views:
+
+1.  The **job view** to upload, preview, and submit Stratosphere programs.
+2.  The **plan view** to analyze the optimized execution plans of Stratosphere programs.
+
+## Job View
+
+The interface starts serving the job view. 
+
+You can **upload** a Stratosphere program as a jar file. To **execute** an uploaded program:
+
+* select it from the job list on the left, 
+* enter the program arguments in the *"Arguments"* field (bottom left), and 
+* click on the *"Run Job"* button (bottom right).
+
+If the *“Show optimizer plan”* option is enabled (default), the *plan view* is display next, otherwise the job is directly submitted to the JobManager for execution.
+
+In case the jar's manifest file does not specify the program class, you can specify it before the argument list as:
+
+```
+assembler <assemblerClass> <programArgs...>
+```
+
+## Plan View
+
+The plan view shows the optimized execution plan of the submitted program in the upper half of the page. The bottom part of the page displays detailed information about the currently selected plan operator including:
+
+* the chosen shipping strategies (local forward, hash partition, range partition, broadcast, ...),
+* the chosen local strategy (sort, hash join, merge join, ...),
+* inferred data properties (partitioning, grouping, sorting), and 
+* used optimizer estimates (data size, I/O and network costs, ...).
+
+To submit the job for execution, click again on the *"Run Job"* button in the bottom right.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md
new file mode 100644
index 0000000..c317e06
--- /dev/null
+++ b/docs/yarn_setup.md
@@ -0,0 +1,188 @@
+---
+title:  "YARN Setup"
+---
+
+# In a Nutshell
+
+Start YARN session with 4 Taskmanagers (each with 4 GB of Heapspace):
+
+```bash
+wget {{ site.docs_05_yarn_archive }}
+tar xvzf stratosphere-dist-{{ site.docs_05_stable }}-yarn.tar.gz
+cd stratosphere-yarn-{{ site.docs_05_stable }}/
+./bin/yarn-session.sh -n 4 -jm 1024 -tm 4096
+```
+
+# Introducing YARN
+
+Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management framework. It allows to run various distributed applications on top of a cluster. Stratosphere runs on YARN next to other applications. Users do not have to setup or install anything if there is already a YARN setup.
+
+**Requirements**
+
+- Apache Hadoop 2.2
+- HDFS
+
+If you have troubles using the Stratosphere YARN client, have a look in the [FAQ section]({{site.baseurl}}/docs/0.5/general/faq.html).
+
+## Start Stratosphere Session
+
+Follow these instructions to learn how to launch a Stratosphere Session within your YARN cluster.
+
+A session will start all required Stratosphere services (JobManager and TaskManagers) so that you can submit programs to the cluster. Note that you can run multiple programs per session.
+
+### Download Stratosphere for YARN
+
+Download the YARN tgz package on the [download page]({{site.baseurl}}/downloads/#nightly). It contains the required files.
+
+
+If you want to build the YARN .tgz file from sources, follow the build instructions. Make sure to use the `-Dhadoop.profile=2` profile. You can find the file in `stratosphere-dist/target/stratosphere-dist-{{site.docs_05_stable}}-yarn.tar.gz` (*Note: The version might be different for you* ).
+
+Extract the package using:
+
+```bash
+tar xvzf stratosphere-dist-{{site.docs_05_stable}}-yarn.tar.gz
+cd stratosphere-yarn-{{site.docs_05_stable}}/
+```
+
+### Start a Session
+
+Use the following command to start a session
+
+```bash
+./bin/yarn-session.sh
+```
+
+This command will show you the following overview:
+
+```bash
+Usage:
+   Required
+     -n,--container <arg>   Number of Yarn container to allocate (=Number of TaskTrackers)
+   Optional
+     -jm,--jobManagerMemory <arg>    Memory for JobManager Container [in MB]
+     -q,--query                      Display available YARN resources (memory, cores)
+     -qu,--queue <arg>               Specify YARN queue.
+     -tm,--taskManagerMemory <arg>   Memory per TaskManager Container [in MB]
+     -tmc,--taskManagerCores <arg>   Virtual CPU cores per TaskManager
+     -v,--verbose                    Verbose debug mode
+```
+
+Please note that the Client requires the `HADOOP_HOME` (or `YARN_CONF_DIR` or `HADOOP_CONF_DIR`) environment variable to be set to read the YARN and HDFS configuration.
+
+**Example:** Issue the following command to allocate 10 TaskTrackers, with 8 GB of memory each:
+
+```bash
+./bin/yarn-session.sh -n 10 -tm 8192
+```
+
+The system will use the configuration in `conf/stratosphere-config.yaml`. Please follow our [configuration guide]({{site.baseurl}}/docs/0.5/setup/config.html) if you want to change something. Stratosphere on YARN will overwrite the following configuration parameters `jobmanager.rpc.address` (because the JobManager is always allocated at different machines) and `taskmanager.tmp.dirs` (we are using the tmp directories given by YARN).
+
+The example invocation starts 11 containers, since there is one additional container for the ApplicationMaster and JobTracker.
+
+Once Stratosphere is deployed in your YARN cluster, it will show you the connection details of the JobTracker.
+
+The client has to remain open to keep the deployment running. We suggest to use `screen`, which will start a detachable shell:
+
+1. Open `screen`,
+2. Start Stratosphere on YARN,
+3. Use `CTRL+a`, then press `d` to detach the screen session,
+4. Use `screen -r` to resume again.
+
+# Submit Job to Stratosphere
+
+Use the following command to submit a Stratosphere program to the YARN cluster:
+
+```bash
+./bin/stratosphere
+```
+
+Please refer to the documentation of the [commandline client]({{site.baseurl}}/docs/0.5/program_execution/cli_client.html).
+
+The command will show you a help menu like this:
+
+```bash
+[...]
+Action "run" compiles and submits a Stratosphere program.
+  "run" action arguments:
+     -a,--arguments <programArgs>   Program arguments
+     -c,--class <classname>         Program class
+     -j,--jarfile <jarfile>         Stratosphere program JAR file
+     -m,--jobmanager <host:port>    Jobmanager to which the program is submitted
+     -w,--wait                      Wait for program to finish
+[...]
+```
+
+Use the *run* action to submit a job to YARN. The client is able to determine the address of the JobManager. In the rare event of a problem, you can also pass the JobManager address using the `-m` argument. The JobManager address is visible in the YARN console.
+
+**Example**
+
+```bash
+wget -O apache-license-v2.txt http://www.apache.org/licenses/LICENSE-2.0.txt
+
+./bin/stratosphere run -j ./examples/stratosphere-java-examples-{{site.docs_05_stable}}-WordCount.jar \
+                       -a 1 file://`pwd`/apache-license-v2.txt file://`pwd`/wordcount-result.txt 
+```
+
+If there is the following error, make sure that all TaskManagers started:
+
+```bash
+Exception in thread "main" eu.stratosphere.compiler.CompilerException:
+    Available instances could not be determined from job manager: Connection timed out.
+```
+
+You can check the number of TaskManagers in the JobManager web interface. The address of this interface is printed in the YARN session console.
+
+If the TaskManagers do not show up after a minute, you should investigate the issue using the log files.
+
+# Build Stratosphere for a specific Hadoop Version
+
+This section covers building Stratosphere for a specific Hadoop version. Most users do not need to do this manually.
+The problem is that Stratosphere uses HDFS and YARN which are both from Apache Hadoop. There exist many different builds of Hadoop (from both the upstream project and the different Hadoop distributions). Typically errors arise with the RPC services. An error could look like this:
+
+```
+ERROR: The job was not successfully submitted to the nephele job manager:
+    eu.stratosphere.nephele.executiongraph.GraphConversionException: Cannot compute input splits for TSV:
+    java.io.IOException: Failed on local exception: com.google.protobuf.InvalidProtocolBufferException:
+    Protocol message contained an invalid tag (zero).; Host Details :
+```
+
+**Example**
+
+```
+mvn -Dhadoop.profile=2 -Pcdh-repo -Dhadoop.version=2.2.0-cdh5.0.0-beta-2 -DskipTests package
+```
+
+The commands in detail:
+
+*  `-Dhadoop.profile=2` activates the Hadoop YARN profile of Stratosphere. This will enable all components of Stratosphere that are compatible with Hadoop 2.2
+*  `-Pcdh-repo` activates the Cloudera Hadoop dependencies. If you want other vendor's Hadoop dependencies (not in maven central) add the repository to your local maven configuration in `~/.m2/`.
+* `-Dhadoop.version=2.2.0-cdh5.0.0-beta-2` sets a special version of the Hadoop dependencies. Make sure that the specified Hadoop version is compatible with the profile you activated.
+
+If you want to build HDFS for Hadoop 2 without YARN, use the following parameter:
+
+```
+-P!include-yarn
+```
+
+Some Cloudera versions (such as `2.0.0-cdh4.2.0`) require this, since they have a new HDFS version with the old YARN API.
+
+Please post to the [Stratosphere mailinglist](https://groups.google.com/d/forum/stratosphere-dev) or create an issue on [Github](https://github.com/stratosphere/stratosphere/issues), if you have issues with your YARN setup and Stratosphere.
+
+# Background
+
+This section briefly describes how Stratosphere and YARN interact. 
+
+<img src="{{site.baseurl}}/img/StratosphereOnYarn.svg" class="img-responsive">
+
+The YARN client needs to access the Hadoop configuration to connect to the YARN resource manager and to HDFS. It determines the Hadoop configuration using the following strategy:
+
+* Test if `YARN_CONF_DIR`, `HADOOP_CONF_DIR` or `HADOOP_CONF_PATH` are set (in that order). If one of these variables are set, they are used to read the configuration.
+* If the above strategy fails (this should not be the case in a correct YARN setup), the client is using the `HADOOP_HOME` environment variable. If it is set, the client tries to access `$HADOOP_HOME/etc/hadoop` (Hadoop 2) and `$HADOOP_HOME/conf` (Hadoop 1).
+
+When starting a new Stratosphere YARN session, the client first checks if the requested resources (containers and memory) are available. After that, it uploads a jar that contains Stratosphere and the configuration to HDFS (step 1).
+
+The next step of the client is to request (step 2) a YARN container to start the *ApplicationMaster* (step 3). Since the client registered the configuration and jar-file as a resource for the container, the NodeManager of YARN running on that particular machine will take care of preparing the container (e.g. downloading the files). Once that has finished, the *ApplicationMaster* (AM) is started.
+
+The *JobManager* and AM are running in the same container. Once they successfully started, the AM knows the address of the JobManager (its own host). It is generating a new Stratosphere configuration file for the TaskManagers (so that they can connect to the JobManager). The file is also uploaded to HDFS. Additionally, the *AM* container is also serving Stratosphere's web interface.
+
+After that, the AM starts allocating the containers for Stratosphere's TaskManagers, which will download the jar file and the modified configuration from the HDFS. Once these steps are completed, Stratosphere is set up and ready to accept Jobs.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 96acd1d..4c29132 100644
--- a/pom.xml
+++ b/pom.xml
@@ -368,6 +368,8 @@
 						<exclude>**/*.iml</exclude>
 						<!-- Generated content -->
 						<exclude>**/target/**</exclude>
+						<!-- Documentation -->
+						<exclude>**/docs/**</exclude>
 					</excludes>
 				</configuration>
 			</plugin>


[3/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/java_api_examples.md b/docs/java_api_examples.md
new file mode 100644
index 0000000..ddb43e6
--- /dev/null
+++ b/docs/java_api_examples.md
@@ -0,0 +1,304 @@
+---
+title:  "Java API Examples"
+---
+
+The following example programs showcase different applications of Stratosphere 
+from simple word counting to graph algorithms. The code samples illustrate the 
+use of **[Stratosphere's Java API]({{site.baseurl}}/docs/{{site current_stable}}/programming_guides/java.html)**. 
+
+The full source code of the following and more examples can be found in the **[stratosphere-java-examples](https://github.com/stratosphere/stratosphere/tree/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples)** module.
+
+# Word Count
+WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
+
+```java
+// get input data
+DataSet<String> text = getTextDataSet(env);
+
+DataSet<Tuple2<String, Integer>> counts = 
+        // split up the lines in pairs (2-tuples) containing: (word,1)
+        text.flatMap(new Tokenizer())
+        // group by the tuple field "0" and sum up tuple field "1"
+        .groupBy(0)
+        .aggregate(Aggregations.SUM, 1);
+
+counts.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+public static final class Tokenizer extends FlatMapFunction<String, Tuple2<String, Integer>> {
+
+    @Override
+    public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+        // normalize and split the line
+        String[] tokens = value.toLowerCase().split("\\W+");
+        
+        // emit the pairs
+        for (String token : tokens) {
+            if (token.length() > 0) {
+                out.collect(new Tuple2<String, Integer>(token, 1));
+            }   
+        }
+    }
+}
+```
+
+The [WordCount example](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java) implements the above described algorithm with input parameters: `<text input path>, <output path>`. As test data, any text file will do.
+
+# Page Rank
+
+The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
+
+In this simple example, PageRank is implemented with a [bulk iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations) and a fixed number of iterations.
+
+```java
+// get input data
+DataSet<Tuple2<Long, Double>> pagesWithRanks = getPagesWithRanksDataSet(env);
+DataSet<Tuple2<Long, Long[]>> pageLinkLists = getLinksDataSet(env);
+
+// set iterative data set
+IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(maxIterations);
+
+DataSet<Tuple2<Long, Double>> newRanks = iteration
+        // join pages with outgoing edges and distribute rank
+        .join(pageLinkLists).where(0).equalTo(0).flatMap(new JoinVertexWithEdgesMatch())
+        // collect and sum ranks
+        .groupBy(0).aggregate(SUM, 1)
+        // apply dampening factor
+        .map(new Dampener(DAMPENING_FACTOR, numPages));
+
+DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
+        newRanks, 
+        newRanks.join(iteration).where(0).equalTo(0)
+        // termination condition
+        .filter(new EpsilonFilter()));
+
+finalPageRanks.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class JoinVertexWithEdgesMatch 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>>, 
+                                            Tuple2<Long, Double>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Long[]>> value, 
+                        Collector<Tuple2<Long, Double>> out) {
+        Long[] neigbors = value.f1.f1;
+        double rank = value.f0.f1;
+        double rankToDistribute = rank / ((double) neigbors.length);
+            
+        for (int i = 0; i < neigbors.length; i++) {
+            out.collect(new Tuple2<Long, Double>(neigbors[i], rankToDistribute));
+        }
+    }
+}
+
+public static final class Dampener extends MapFunction<Tuple2<Long,Double>, Tuple2<Long,Double>> {
+    private final double dampening, randomJump;
+
+    public Dampener(double dampening, double numVertices) {
+        this.dampening = dampening;
+        this.randomJump = (1 - dampening) / numVertices;
+    }
+
+    @Override
+    public Tuple2<Long, Double> map(Tuple2<Long, Double> value) {
+        value.f1 = (value.f1 * dampening) + randomJump;
+        return value;
+    }
+}
+
+public static final class EpsilonFilter 
+                    extends FilterFunction<Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>>> {
+
+    @Override
+    public boolean filter(Tuple2<Tuple2<Long, Double>, Tuple2<Long, Double>> value) {
+        return Math.abs(value.f0.f1 - value.f1.f1) > EPSILON;
+    }
+}
+```
+
+The [PageRank program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/PageRankBasic.java) implements the above example.
+It requires the following parameters to run: `<pages input path>, <links input path>, <output path>, <num pages>, <num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Pages represented as an (long) ID separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
+
+For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+
+# Connected Components
+
+The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
+
+This implementation uses a [delta iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations): Vertices that have not changed their component ID do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
+
+```java
+// read vertex and edge data
+DataSet<Long> vertices = getVertexDataSet(env);
+DataSet<Tuple2<Long, Long>> edges = getEdgeDataSet(env).flatMap(new UndirectEdge());
+
+// assign the initial component IDs (equal to the vertex ID)
+DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
+        
+// open a delta iteration
+DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+        verticesWithInitialId.iterateDelta(verticesWithInitialId, maxIterations, 0);
+
+// apply the step logic: 
+DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset()
+        // join with the edges
+        .join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
+        // select the minimum neighbor component ID
+        .groupBy(0).aggregate(Aggregations.MIN, 1)
+        // update if the component ID of the candidate is smaller
+        .join(iteration.getSolutionSet()).where(0).equalTo(0)
+        .flatMap(new ComponentIdFilter());
+
+// close the delta iteration (delta and new workset are identical)
+DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
+
+// emit result
+result.writeAsCsv(outputPath, "\n", " ");
+
+// User-defined functions
+
+public static final class DuplicateValue<T> extends MapFunction<T, Tuple2<T, T>> {
+    
+    @Override
+    public Tuple2<T, T> map(T vertex) {
+        return new Tuple2<T, T>(vertex, vertex);
+    }
+}
+
+public static final class UndirectEdge 
+                    extends FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+    Tuple2<Long, Long> invertedEdge = new Tuple2<Long, Long>();
+    
+    @Override
+    public void flatMap(Tuple2<Long, Long> edge, Collector<Tuple2<Long, Long>> out) {
+        invertedEdge.f0 = edge.f1;
+        invertedEdge.f1 = edge.f0;
+        out.collect(edge);
+        out.collect(invertedEdge);
+    }
+}
+
+public static final class NeighborWithComponentIDJoin 
+                    extends JoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+    @Override
+    public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithComponent, Tuple2<Long, Long> edge) {
+        return new Tuple2<Long, Long>(edge.f1, vertexWithComponent.f1);
+    }
+}
+
+public static final class ComponentIdFilter 
+                    extends FlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, 
+                                            Tuple2<Long, Long>> {
+
+    @Override
+    public void flatMap(Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> value, 
+                        Collector<Tuple2<Long, Long>> out) {
+        if (value.f0.f1 < value.f1.f1) {
+            out.collect(value.f0);
+        }
+    }
+}
+```
+
+The [ConnectedComponents program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/graph/ConnectedComponents.java) implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Vertices represented as IDs and separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
+- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+
+# Relational Query
+
+The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
+
+The example implements the following SQL query.
+
+```sql
+SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+    FROM orders, lineitem
+WHERE l_orderkey = o_orderkey
+    AND o_orderstatus = "F" 
+    AND YEAR(o_orderdate) > 1993
+    AND o_orderpriority LIKE "5%"
+GROUP BY l_orderkey, o_shippriority;
+```
+
+The Stratosphere Java program, which implements the above query looks as follows.
+
+```java
+// get orders data set: (orderkey, orderstatus, orderdate, orderpriority, shippriority)
+DataSet<Tuple5<Integer, String, String, String, Integer>> orders = getOrdersDataSet(env);
+// get lineitem data set: (orderkey, extendedprice)
+DataSet<Tuple2<Integer, Double>> lineitems = getLineitemDataSet(env);
+
+// orders filtered by year: (orderkey, custkey)
+DataSet<Tuple2<Integer, Integer>> ordersFilteredByYear =
+        // filter orders
+        orders.filter(
+            new FilterFunction<Tuple5<Integer, String, String, String, Integer>>() {
+                @Override
+                public boolean filter(Tuple5<Integer, String, String, String, Integer> t) {
+                    // status filter
+                    if(!t.f1.equals(STATUS_FILTER)) {
+                        return false;
+                    // year filter
+                    } else if(Integer.parseInt(t.f2.substring(0, 4)) <= YEAR_FILTER) {
+                        return false;
+                    // order priority filter
+                    } else if(!t.f3.startsWith(OPRIO_FILTER)) {
+                        return false;
+                    }
+                    return true;
+                }
+            })
+        // project fields out that are no longer required
+        .project(0,4).types(Integer.class, Integer.class);
+
+// join orders with lineitems: (orderkey, shippriority, extendedprice)
+DataSet<Tuple3<Integer, Integer, Double>> lineitemsOfOrders = 
+        ordersFilteredByYear.joinWithHuge(lineitems)
+                            .where(0).equalTo(0)
+                            .projectFirst(0,1).projectSecond(1)
+                            .types(Integer.class, Integer.class, Double.class);
+
+// extendedprice sums: (orderkey, shippriority, sum(extendedprice))
+DataSet<Tuple3<Integer, Integer, Double>> priceSums = 
+        // group by order and sum extendedprice
+        lineitemsOfOrders.groupBy(0,1).aggregate(Aggregations.SUM, 2);
+
+// emit result
+priceSums.writeAsCsv(outputPath);
+```
+
+The [Relational Query program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/RelationalQuery.java) implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>`.
+
+The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
+Take the following steps to generate arbitrary large input files for the provided Stratosphere programs:
+
+1.  Download and unpack DBGEN
+2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
+
+```bash
+DATABASE = DB2
+MACHINE  = LINUX
+WORKLOAD = TPCH
+CC       = gcc
+```
+
+1.  Build DBGEN using *make*
+2.  Generate lineitem and orders relations using dbgen. A scale factor
+    (-s) of 1 results in a generated data set with about 1 GB size.
+
+```bash
+./dbgen -T o -s 1
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/java_api_guide.md b/docs/java_api_guide.md
new file mode 100644
index 0000000..c09c26a
--- /dev/null
+++ b/docs/java_api_guide.md
@@ -0,0 +1,1476 @@
+---
+title: "Java API Programming Guide"
+---
+
+Java API
+========
+
+<section id="introduction">
+Introduction
+------------
+
+Analysis programs in Stratosphere are regular Java programs that implement transformations on data sets (e.g., filtering, mapping, joining, grouping). The data sets are initially created from certain sources (e.g., by reading files, or from collections). Results are returned via sinks, which may for example write the data to (distributed) files, or to standard output (for example the command line terminal). Stratosphere programs run in a variety of contexts, standalone, or embedded in other programs. The execution can happen in a local JVM, or on clusters of many machines.
+
+In order to create your own Stratosphere program, we encourage you to start with the [program skeleton](#skeleton) and gradually add your own [transformations](#transformations). The remaining sections act as references for additional operations and advanced features.
+</section>
+
+<section id="toc">
+
+<div id="docs_05_toc">
+  <div class="list-group">
+{% for sublink in page.toc %}
+   <a href="#{{ sublink.anchor }}" class="list-group-item">{{forloop.index}}. <strong>{{ sublink.title }}</strong></a>
+{% endfor %}
+  </div>
+</div>
+
+</section>
+
+<section id="example">
+Example Program
+---------------
+
+The following program is a complete, working example of WordCount. You can copy &amp; paste the code to run it locally. You only have to include Stratosphere's Java API library into your project (see Section [Linking with Stratosphere](#linking)) and specify the imports. Then you are ready to go!
+
+```java
+public class WordCountExample {
+    public static void main(String[] args) throws Exception {
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        DataSet<String> text = env.fromElements(
+            "Who's there?",
+            "I think I hear them. Stand, ho! Who's there?");
+
+        DataSet<Tuple2<String, Integer>> wordCounts = text
+            .flatMap(new LineSplitter())
+            .groupBy(0)
+            .aggregate(Aggregations.SUM, 1);
+
+        wordCounts.print();
+
+        env.execute("Word Count Example");
+    }
+
+    public static final class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
+        @Override
+        public void flatMap(String line, Collector<Tuple2<String, Integer>> out) {
+            for (String word : line.split(" ")) {
+                out.collect(new Tuple2<String, Integer>(word, 1));
+            }
+        }
+    }
+}
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="linking">
+Linking with Stratosphere
+-------------------------
+
+To write programs with Stratosphere, you need to include Stratosphere’s Java API library in your project.
+
+The simplest way to do this is to use the [quickstart scripts]({{site.baseurl}}/quickstart/java.html). They create a blank project from a template (a Maven Archetype), which sets up everything for you. To manually create the project, you can use the archetype and create a project by calling:
+
+{% highlight bash %}
+mvn archetype:generate /
+    -DarchetypeGroupId=eu.stratosphere /
+    -DarchetypeArtifactId=quickstart-java /
+    -DarchetypeVersion={{site.docs_05_stable}}
+{% endhighlight %}
+
+If you want to add Stratosphere to an existing Maven project, add the following entry to your *dependencies* section in the *pom.xml* file of your project:
+
+{% highlight xml %}
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-java</artifactId>
+  <version>{{site.docs_05_stable}}</version>
+</dependency>
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{site.docs_05_stable}}</version>
+</dependency>
+{% endhighlight %}
+
+In order to link against the latest SNAPSHOT versions of the code, please follow [this guide]({{site.baseurl}}/downloads/#nightly).
+
+The *stratosphere-clients* dependency is only necessary to invoke the Stratosphere program locally (for example to run it standalone for testing and debugging). 
+If you intend to only export the program as a JAR file and [run it on a cluster]({{site.baseurl}}/docs/0.5/program_execution/cluster_execution.html), you can skip that dependency.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="skeleton">
+Program Skeleton
+----------------
+
+As we already saw in the example, Stratosphere programs look like regular Java
+programs with a `main()` method. Each program consists of the same basic parts:
+
+1. Obtain an `ExecutionEnvironment`,
+2. Load/create the initial data,
+3. Specify transformations on this data,
+4. Specify where to put the results of your computations, and
+5. Execute your program.
+
+We will now give an overview of each of those steps but please refer
+to the respective sections for more details. Note that all [core classes
+of the Java API](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java) are found in the package `eu.stratosphere.api.java`.
+
+The `ExecutionEnvironment` is the basis for all Stratosphere programs. You can
+obtain one using these static methods on class `ExecutionEnvironment`:
+
+```java
+getExecutionEnvironment()
+
+createLocalEnvironment()
+createLocalEnvironment(int degreeOfParallelism)
+
+createRemoteEnvironment(String host, int port, String... jarFiles)
+createRemoteEnvironment(String host, int port, int degreeOfParallelism, String... jarFiles)
+```
+
+Typically, you only need to use `getExecutionEnvironment()`, since this
+will do the right thing depending on the context: if you are executing
+your program inside an IDE or as a regular Java program it will create
+a local environment that will execute your program on your local machine. If
+you created a JAR file from you program, and invoke it through the [command line]({{site.baseurl}}/docs/0.5/program_execution/cli_client.html)
+or the [web interface]({{site.baseurl}}/docs/0.5/program_execution/web_interface.html),
+the Stratosphere cluster manager will
+execute your main method and `getExecutionEnvironment()` will return
+an execution environment for executing your program on a cluster.
+
+For specifying data sources the execution environment has several methods
+to read from files using various methods: you can just read them line by line,
+as CSV files, or using completely custom data input formats. To just read
+a text file as a sequence of lines, you could use:
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+DataSet<String> text = env.readTextFile("file:///path/to/file");
+```
+
+This will give you a `DataSet` on which you can then apply transformations. For
+more information on data sources and input formats, please refer to
+[Data Sources](#data_sources).
+
+Once you have a `DataSet` you can apply transformations to create a new
+`DataSet` which you can then write to a file, transform again, or
+combine with other `DataSet`s. You apply transformations by calling
+methods on `DataSet` with your own custom transformation function. For example,
+map looks like this:
+
+```java
+DataSet<String> input = ...;
+
+DataSet<Integer> tokenized = text.map(new MapFunction<String, Integer>() {
+    @Override
+    public Integer map(String value) {
+        return Integer.parseInt(value);
+    }
+});
+```
+
+This will create a new `DataSet` by converting every String in the original
+set to an Integer. For more information and a list of all the transformations,
+please refer to [Transformations](#transformations).
+
+Once you have a `DataSet` that needs to be written to disk you call one
+of these methods on `DataSet`:
+
+```java
+writeAsText(String path)
+writeAsCsv(String path)
+write(FileOutputFormat<T> outputFormat, String filePath)
+
+print()
+```
+
+The last method is only useful for developing/debugging on a local machine,
+it will output the contents of the `DataSet` to standard output. (Note that in
+a cluster, the result goes to the standard out stream of the cluster nodes and ends
+up in the *.out* files of the workers).
+The first two do as the name suggests, the third one can be used to specify a
+custom data output format. Keep in mind, that these calls do not actually
+write to a file yet. Only when your program is completely specified and you
+call the `execute` method on your `ExecutionEnvironment` are all the
+transformations executed and is data written to disk. Please refer
+to [Data Sinks](#data_sinks) for more information on writing to files and also
+about custom data output formats.
+
+Once you specified the complete program you need to call `execute` on
+the `ExecutionEnvironment`. This will either execute on your local
+machine or submit your program for execution on a cluster, depending on
+how you created the execution environment.
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="lazyeval">
+Lazy Evaluation
+---------------
+
+All Stratosphere programs are executed lazily: When the program's main method is executed, the data loading and transformations do not happen directly. Rather, each operation is created and added to the program's plan. The operations are actually executed when one of the `execute()` methods is invoked on the ExecutionEnvironment object. Whether the program is executed locally or on a cluster depends on the environment of the program.
+
+The lazy evaluation lets you construct sophisticated programs that Stratosphere executes as one holistically planned unit.
+</section>
+
+<section id="types">
+Data Types
+----------
+
+The Java API is strongly typed: All data sets and transformations accept typed elements. This catches type errors very early and supports safe refactoring of programs. The API supports various different data types for the input and output of operators. Both `DataSet` and functions like `MapFunction`, `ReduceFunction`, etc. are parameterized with data types using Java generics in order to ensure type-safety.
+
+There are four different categories of data types, which are treated slightly different:
+
+1. **Regular Types**
+2. **Tuples**
+3. **Values**
+4. **Hadoop Writables**
+
+
+#### Regular Types
+
+Out of the box, the Java API supports all common basic Java types: `Byte`, `Short`, `Integer`, `Long`, `Float`, `Double`, `Boolean`, `Character`, `String`.
+
+Furthermore, you can use the vast majority of custom Java classes. Restrictions apply to classes containing fields that cannot be serialized, like File pointers, I/O streams, or other native resources. Classes that follow the Java Beans conventions work well in general. The following defines a simple example class to illustrate how you can use custom classes:
+
+```java
+public class WordWithCount {
+
+    public String word;
+    public int count;
+
+    public WordCount() {}
+
+    public WordCount(String word, int count) {
+        this.word = word;
+        this.count = count;
+    }
+}
+```
+
+You can use all of those types to parameterize `DataSet` and function implementations, e.g. `DataSet<String>` for a `String` data set or `MapFunction<String, Integer>` for a mapper from `String` to `Integer`.
+
+```java
+
+// using a basic data type
+DataSet<String> numbers = env.fromElements("1", "2");
+
+numbers.map(new MapFunction<String, Integer>() {
+    @Override
+    public String map(String value) throws Exception {
+        return Integer.parseInt(value);
+    }
+});
+
+// using a custom class
+DataSet<WordCount> wordCounts = env.fromElements(
+    new WordCount("hello", 1),
+    new WordCount("world", 2));
+
+wordCounts.map(new MapFunction<WordCount, Integer>() {
+    @Override
+    public String map(WordCount value) throws Exception {
+        return value.count;
+    }
+});
+```
+
+When working with operators that require a Key for grouping or matching records
+you need to implement a `KeySelector` for your custom type (see
+[Section Data Transformations](#transformations)).
+
+```java
+wordCounts.groupBy(new KeySelector<WordCount, String>() {
+    public String getKey(WordCount v) {
+        return v.word;
+    }
+}).reduce(new MyReduceFunction());
+```
+
+#### Tuples
+
+You can use the `Tuple` classes for composite types. Tuples contain a fix number of fields of various types. The Java API provides classes from `Tuple1` up to `Tuple25`. Every field of a tuple can be an arbitrary Stratosphere type - including further tuples, resulting in nested tuples. Fields of a Tuple can be accessed directly using the fields `tuple.f4`, or using the generic getter method `tuple.getField(int position)`. The field numbering starts with 0. Note that this stands in contrast to the Scala tuples, but it is more consistent with Java's general indexing.
+
+```java
+DataSet<Tuple2<String, Integer>> wordCounts = env.fromElements(
+    new Tuple2<String, Integer>("hello", 1),
+    new Tuple2<String, Integer>("world", 2));
+
+wordCounts.map(new MapFunction<Tuple2<String, Integer>, Integer>() {
+    @Override
+    public String map(Tuple2<String, Integer> value) throws Exception {
+        return value.f1;
+    }
+});
+```
+
+When working with operators that require a Key for grouping or matching records,
+Tuples let you simply specify the positions of the fields to be used as key. You can specify more
+than one position to use composite keys (see [Section Data Transformations](#transformations)).
+
+```java
+wordCounts
+    .groupBy(0)
+    .reduce(new MyReduceFunction());
+```
+
+In order to access fields more intuitively and to generate more readable code, it is also possible to extend a subclass of `Tuple`. You can add getters and setters with custom names that delegate to the field positions. See this [example](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/relational/TPCHQuery3.java) for an illustration how to make use of that mechanism.
+
+
+#### Values
+
+*Value* types describe their serialization and deserialization manually. Instead of going through a general purpose serialization framework, they provide custom code for those operations by means implementing the `eu.stratosphere.types.Value` interface with the methods `read` and `write`. Using a *Value* type is reasonable when general purpose serialization would be highly inefficient. An example would be a data type that implements a sparse vector of elements as an array. Knowing that the array is mostly zero, one can use a special encoding for the non-zero elements, while the general purpose serialization would simply write all array elements.
+
+The `eu.stratosphere.types.CopyableValue` interface supports manual internal cloning logic in a similar way.
+
+Stratosphere comes with pre-defined Value types that correspond to Java's basic data types. (`ByteValue`, `ShortValue`, `IntValue`, `LongValue`, `FloatValue`, `DoubleValue`, `StringValue`, `CharValue`, `BooleanValue`). These Value types act as mutable variants of the basic data types: Their value can be altered, allowing programmers to reuse objects and take pressure off the garbage collector. 
+
+
+#### Hadoop Writables
+
+You can use types that implement the `org.apache.hadoop.Writable` interface. The serialization logic defined in the `write()`and `readFields()` methods will be used for serialization.
+
+
+#### Type Erasure & Type Inferrence
+
+The Java compiler throws away much of the generic type information after the compilation. This is known as *type erasure* in Java. It means that at runtime, an instance of an object does not know its generic type any more. For example, instances of `DataSet<String>` and `DataSet<Long>` look the same to the JVM.
+
+Stratosphere requires type information at the time when it prepares the program for execution (when the main method of the program is called). The Stratosphere Java API tries to reconstruct the type information that was thrown away in various ways and store it explicitly in the data sets and operators. You can retrieve the type via `DataSet.getType()`. The method returns an instance of `TypeInformation`, which is Stratosphere's internal way of representing types.
+
+The type inference has its limits and needs the "cooperation" of the programmer in some cases. Examples for that are methods that create data sets from collections, such as `ExecutionEnvironment.fromCollection(),` where you can pass an argument that describes the type. But also generic functions like `MapFunction<I, O>` may need extra type information.
+
+The [ResultTypeQueryable](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/ResultTypeQueryable.java) interface can be implemented by input formats and functions to tell the API explicitly about their return type. The *input types* that the functions are invoked with can usually be inferred by the result types of the previous operations.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="transformations">
+Data Transformations
+--------------------
+
+A data transformation transforms one or more `DataSet`s into a new `DataSet`. Advanced data analysis programs can be assembled by chaining multiple transformations.
+
+### Map
+
+The Map transformation applies a user-defined `MapFunction` on each element of a DataSet.
+It implements a one-to-one mapping, that is, exactly one element must be returned by
+the function.
+
+The following code transforms a `DataSet` of Integer pairs into a `DataSet` of Integers:
+
+```java
+// MapFunction that adds two integer values
+public class IntAdder extends MapFunction<Tuple2<Integer, Integer>, Integer> {
+  @Override
+  public Integer map(Tuple2<Integer, Integer> in) {
+    return in.f0 + in.f1;
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, Integer>> intPairs = // [...]
+DataSet<Integer> intSums = intPairs.map(new IntAdder());
+```
+
+### FlatMap
+
+The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataSet`.
+This variant of a map function can return arbitrary many result elements (including none) for each input element.
+
+The following code transforms a `DataSet` of text lines into a `DataSet` of words:
+
+```java
+// FlatMapFunction that tokenizes a String by whitespace characters and emits all String tokens.
+public class Tokenizer extends FlatMapFunction<String, String> {
+  @Override
+  public void flatMap(String value, Collector<String> out) {
+    for (String token : value.split("\\W")) {
+      out.collect(token);
+    }
+  }
+}
+
+// [...]
+DataSet<String> textLines = // [...]
+DataSet<String> words = textLines.flatMap(new Tokenizer());
+
+```
+
+### Filter
+
+The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataSet` and retains only those elements for which the function returns `true`.
+
+The following code removes all Integers smaller than zero from a `DataSet`:
+
+```java
+// FilterFunction that filters out all Integers smaller than zero.
+public class NaturalNumberFilter extends FilterFunction<Integer> {
+  @Override
+  public boolean filter(Integer number) {
+    return number >= 0;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> naturalNumbers = intNumbers.filter(new NaturalNumberFilter());
+```
+
+### Project (Tuple DataSets only)
+
+The Project transformation removes or moves `Tuple` fields of a `Tuple` `DataSet`.
+The `project(int...)` method selects `Tuple` fields that should be retained by their index and defines their order in the output `Tuple`.
+The `types(Class<?> ...)`method must give the types of the output `Tuple` fields.
+
+Projections do not require the definition of a user function.
+
+The following code shows different ways to apply a Project transformation on a `DataSet`:
+
+```java
+DataSet<Tuple3<Integer, Double, String>> in = // [...]
+// converts Tuple3<Integer, Double, String> into Tuple2<String, Integer>
+DataSet<Tuple2<String, Integer>> out = in.project(2,0).types(String.class, Integer.class);
+```
+
+### Transformations on grouped DataSet
+
+The reduce operations can operate on grouped data sets. Specifying the key to
+be used for grouping can be done in two ways:
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+Please look at the reduce examples to see how the grouping keys are specified.
+
+### Reduce on grouped DataSet
+
+A Reduce transformation that is applied on a grouped `DataSet` reduces each group to a single element using a user-defined `ReduceFunction`.
+For each group of input elements, a `ReduceFunction` successively combines pairs of elements into one element until only a single element for each group remains.
+
+#### Reduce on DataSet grouped by KeySelector Function
+
+A `KeySelector` function extracts a key value from each element of a `DataSet`. The extracted key value is used to group the `DataSet`.
+The following code shows how to group a POJO `DataSet` using a `KeySelector` function and to reduce it with a `ReduceFunction`.
+
+```java
+// some ordinary POJO
+public class WC {
+  public String word;
+  public int count;
+  // [...]
+}
+
+// ReduceFunction that sums Integer attributes of a POJO
+public class WordCounter extends ReduceFunction<WC> {
+  @Override
+  public WC reduce(WC in1, WC in2) {
+    return new WC(in1.word, in1.count + in2.count);
+  }
+}
+
+// [...]
+DataSet<WC> words = // [...]
+DataSet<WC> wordCounts = words
+                         // DataSet grouping with inline-defined KeySelector function
+                         .groupBy(
+                           new KeySelector<WC, String>() {
+                             public String getKey(WC wc) { return wc.word; }
+                           })
+                         // apply ReduceFunction on grouped DataSet
+                         .reduce(new WordCounter());
+```
+
+#### Reduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+Field position keys specify one or more fields of a `Tuple` `DataSet` that are used as grouping keys.
+The following code shows how to use field position keys and apply a `ReduceFunction`.
+
+```java
+DataSet<Tuple3<String, Integer, Double>> tuples = // [...]
+DataSet<Tuple3<String, Integer, Double>> reducedTuples =
+                                         tuples
+                                         // group DataSet on first and second field of Tuple
+                                         .groupBy(0,1)
+                                         // apply ReduceFunction on grouped DataSet
+                                         .reduce(new MyTupleReducer());
+```
+
+### GroupReduce on grouped DataSet
+
+A GroupReduce transformation that is applied on a grouped `DataSet` calls a user-defined `GroupReduceFunction` for each group. The difference
+between this and `Reduce` is that the user defined function gets the whole group at once.
+The function is invoked with an iterator over all elements of a group and can return an arbitrary number of result elements using the collector.
+
+#### GroupReduce on DataSet grouped by Field Position Keys (Tuple DataSets only)
+
+The following code shows how duplicate strings can be removed from a `DataSet` grouped by Integer.
+
+```java
+public class DistinctReduce
+         extends GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+  // Set to hold all unique strings of a group
+  Set<String> uniqStrings = new HashSet<String>();
+
+  @Override
+  public void reduce(Iterator<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+    // clear set
+    uniqStrings.clear();
+    // there is at least one element in the iterator
+    Tuple2<Integer, String> first = in.next();
+    Integer key = first.f0;
+    uniqStrings.add(first.f1);
+    // add all strings of the group to the set
+    while(in.hasNext()) {
+      uniqStrings.add(in.next().f1);
+    }
+    // emit all unique strings
+    Tuple2<Integer, String> t = new Tuple2<Integer, String>(key, "");
+    for(String s : uniqStrings) {
+      t.f1 = s;
+      out.collect(t);
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Tuple2<Integer, String>> output =
+                                 input
+                                 // group DataSet by the first tuple field
+                                 .groupBy(0)
+                                 // apply GroupReduceFunction on each group and
+                                 //   remove elements with duplicate strings.
+                                 .reduceGroup(new DistinctReduce());
+```
+
+**Note:** Stratosphere internally works a lot with mutable objects. Collecting objects like in the above example only works because Strings are immutable in Java!
+
+#### GroupReduce on DataSet grouped by KeySelector Function
+
+Works analogous to `KeySelector` functions in Reduce transformations.
+
+#### GroupReduce on sorted groups (Tuple DataSets only)
+
+A `GroupReduceFunction` accesses the elements of a group using an iterator. Optionally, the iterator can hand out the elements of a group in a specified order. In many cases this can help to reduce the complexity of a user-defined `GroupReduceFunction` and improve its efficiency.
+Right now, this feature is only available for `Tuple` `DataSet`.
+
+The following code shows another example how to remove duplicate Strings in a `DataSet` grouped by an Integer and sorted by String.
+
+```java
+// GroupReduceFunction that removes consecutive identical elements
+public class DistinctReduce
+         extends GroupReduceFunction<Tuple2<Integer, String>, Tuple2<Integer, String>> {
+  @Override
+  public void reduce(Iterator<Tuple2<Integer, String>> in, Collector<Tuple2<Integer, String>> out) {
+    // there is at least one element in the iterator
+    Tuple2<Integer, String> first = in.next();
+    Integer key = first.f0;
+    String comp = first.f1;
+    // for each element in group
+    while(in.hasNext()) {
+      String next = in.next().f1;
+      // check if strings are different
+      if(!next.equals(comp)) {
+        // emit a new element
+        out.collect(new Tuple2<Integer, String>(key, comp));
+        // update compare string
+        comp = next;
+      }
+    }
+    // emit last element
+    out.collect(new Tuple2<Integer, String>(key, comp));
+  }
+}
+
+// [...]
+DataSet<Tuple2<Integer, String>> input = // [...]
+DataSet<Double> output = input
+                         // group DataSet by the first tuple field
+                         .groupBy(0)
+                         // sort groups on second tuple field
+                         .sortGroup(1, Order.ASCENDING)
+                         // // apply GroupReduceFunction on DataSet with sorted groups
+                         .reduceGroup(new DistinctReduce());
+```
+
+**Note:** A GroupSort often comes for free if the grouping is established using a sort-based execution strategy of an operator before the reduce operation.
+
+#### Combinable GroupReduceFunctions
+
+In contrast to a `ReduceFunction`, a `GroupReduceFunction` is not implicitly combinable. In order to make a `GroupReduceFunction` combinable, you need to implement (override) the ```combine()``` method and annotate the `GroupReduceFunction` with the ```@Combinable``` annotation as shown here:
+
+The following code shows how to compute multiple sums using a combinable `GroupReduceFunction`:
+
+```java
+// Combinable GroupReduceFunction that computes two sums.
+@Combinable
+public class MyCombinableGroupReducer
+         extends GroupReduceFunction<Tuple3<String, Integer, Double>,
+                                     Tuple3<String, Integer, Double>> {
+  @Override
+  public void reduce(Iterator<Tuple3<String, Integer, Double>> in,
+                     Collector<Tuple3<String, Integer, Double>> out) {
+    // one element is always present in iterator
+    Tuple3<String, Integer, Double> curr = in.next();
+    String key = curr.f0;
+    int intSum = curr.f1;
+    double doubleSum = curr.f2;
+    // sum up all ints and doubles
+    while(in.hasNext()) {
+      curr = in.next();
+      intSum += curr.f1;
+      doubleSum += curr.f2;
+    }
+    // emit a tuple with both sums
+    out.collect(new Tuple3<String, Integer, Double>(key, intSum, doubleSum));
+  }
+
+  @Override
+  public void combine(Iterator<Tuple3<String, Integer, Double>> in,
+                      Collector<Tuple3<String, Integer, Double>> out)) {
+    // in some cases combine() calls can simply be forwarded to reduce().
+    this.reduce(in, out);
+  }
+}
+```
+
+### Aggregate on grouped Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet` and supports only field positions keys for grouping.
+
+The following code shows how to apply an Aggregation transformation on a `DataSet` grouped by field position keys:
+
+```java
+DataSet<Tuple3<Integer, String, Double>> input = // [...]
+DataSet<Tuple3<Integer, String, Double>> output = input
+                                          // group DataSet on second field
+                                          .groupBy(1)
+                                          // compute sum of the first field
+                                          .aggregate(SUM, 0)
+                                          // compute minimum of the third field
+                                          .and(MIN, 2);
+```
+
+To apply multiple aggregations on a DataSet it is necessary to use the `.and()` function after the first aggregate, that means `.aggregate(SUM, 0).and(MIN, 2)` produces the sum of field 0 and the minimum of field 2 of the original DataSet. 
+In contrast to that `.aggregate(SUM, 0).aggregate(MIN, 2)` will apply an aggregation on an aggregation. In the given example it would produce the minimum of field 2 after calculating the sum of field 0 grouped by field 1.
+
+**Note:** The set of aggregation functions will be extended in the future.
+
+### Reduce on full DataSet
+
+The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataSet`.
+The `ReduceFunction` subsequently combines pairs of elements into one element until only a single element remains.
+
+The following code shows how to sum all elements of an Integer `DataSet`:
+
+```java
+// ReduceFunction that sums Integers
+public class IntSummer extends ReduceFunction<Integer> {
+  @Override
+  public Integer reduce(Integer num1, Integer num2) {
+    return num1 + num2;
+  }
+}
+
+// [...]
+DataSet<Integer> intNumbers = // [...]
+DataSet<Integer> sum = intNumbers.reduce(new IntSummer());
+```
+
+Reducing a full `DataSet` using the Reduce transformation implies that the final Reduce operation cannot be done in parallel. However, a `ReduceFunction` is automatically combinable such that a Reduce transformation does not limit scalability for most use cases.
+
+### GroupReduce on full DataSet
+
+The GroupReduce transformation applies a user-defined `GroupReduceFunction` on all elements of a `DataSet`.
+A `GroupReduceFunction` can iterate over all elements of `DataSet` and return an arbitrary number of result elements.
+
+The following example shows how to apply a GroupReduce transformation on a full `DataSet`:
+
+```java
+DataSet<Integer> input = // [...]
+// apply a (preferably combinable) GroupReduceFunction to a DataSet
+DataSet<Double> output = input.reduceGroup(new MyGroupReducer());
+```
+
+**Note:** A GroupReduce transformation on a full `DataSet` cannot be done in parallel if the `GroupReduceFunction` is not combinable. Therefore, this can be a very compute intensive operation. See the paragraph on "Combineable `GroupReduceFunction`s" above to learn how to implement a combinable `GroupReduceFunction`.
+
+### Aggregate on full Tuple DataSet
+
+There are some common aggregation operations that are frequently used. The Aggregate transformation provides the following build-in aggregation functions:
+
+- Sum,
+- Min, and
+- Max.
+
+The Aggregate transformation can only be applied on a `Tuple` `DataSet`.
+
+The following code shows how to apply an Aggregation transformation on a full `DataSet`:
+
+```java
+DataSet<Tuple2<Integer, Double>> input = // [...]
+DataSet<Tuple2<Integer, Double>> output = input
+                                          // compute sum of the first field
+                                          .aggregate(SUM, 0)
+                                          // compute minimum of the second field
+                                          .and(MIN, 1);
+```
+
+**Note:** Extending the set of supported aggregation functions is on our roadmap.
+
+### Join
+
+The Join transformation joins two `DataSet`s into one `DataSet`. The elements of both `DataSet`s are joined on one or more keys which can be specified using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+There are a few different ways to perform a Join transformation which are shown in the following.
+
+#### Default Join (Join into Tuple2)
+
+The default Join transformation produces a new `Tuple``DataSet` with two fields. Each tuple holds a joined element of the first input `DataSet` in the first tuple field and a matching element of the second input `DataSet` in the second field.
+
+The following code shows a default Join transformation using field position keys:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Double, Integer>> input2 = // [...]
+// result dataset is typed as Tuple2
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Double, Integer>>>
+            result =
+            input1.join(input2)
+                  // key definition on first DataSet using a field position key
+                  .where(0)
+                  // key definition of second DataSet using a field position key
+                  .equalTo(1);
+```
+
+#### Join with JoinFunction
+
+A Join transformation can also call a user-defined `JoinFunction` to process joining tuples.
+A `JoinFunction` receives one element of the first input `DataSet` and one element of the second input `DataSet` and returns exactly one element.
+
+The following code performs a join of `DataSet` with custom java objects and a `Tuple` `DataSet` using `KeySelector` functions and shows how to call a user-defined `JoinFunction`:
+
+```java
+// some POJO
+public class Rating {
+  public String name;
+  public String category;
+  public int points;
+}
+
+// Join function that joins a custom POJO with a Tuple
+public class PointWeighter
+         extends JoinFunction<Rating, Tuple2<String, Double>, Tuple2<String, Double>> {
+
+  @Override
+  public Tuple2<String, Double> join(Rating rating, Tuple2<String, Double> weight) {
+    // multiply the points and rating and construct a new output tuple
+    return new Tuple2<String, Double>(rating.name, rating.points * weight.f1);
+  }
+}
+
+DataSet<Rating> ratings = // [...]
+DataSet<Tuple2<String, Double>> weights = // [...]
+DataSet<Tuple2<String, Double>>
+            weightedRatings =
+            ratings.join(weights)
+                   // key definition of first DataSet using a KeySelector function
+                   .where(new KeySelection<Rating, String>() {
+                            public String getKey(Rating r) { return r.category; }
+                          })
+                   // key definition of second DataSet using a KeySelector function
+                   .equalTo(new KeySelection<Tuple2<String, Double>, String>() {
+                              public String getKey(Tuple2<String, Double> t) { return t.f0; }
+                            })
+                   // applying the JoinFunction on joining pairs
+                   .with(new PointWeighter());
+```
+
+#### Join with Projection
+
+A Join transformation can construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, String, Double, Byte>
+            result =
+            input1.join(input2)
+                  // key definition on first DataSet using a field position key
+                  .where(0)
+                  // key definition of second DataSet using a field position key
+                  .equalTo(0)
+                  // select and reorder fields of matching tuples
+                  .projectFirst(0,2).projectSecond(1).projectFirst(1)
+                  .types(Integer.class, String.class, Double.class, Byte.class);
+```
+
+`projectFirst(int...)` and `projectSecond(int...)` select the fields of the first and second joined input that should be assembled into an output `Tuple`. The order of indexes defines the order of fields in the output tuple.
+The join projection works also for non-`Tuple` `DataSet`s. In this case, `projectFirst()` or `projectSecond()` must be called without arguments to add a joined element to the output `Tuple`.
+
+#### Join with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to join as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result1 =
+            // hint that the second DataSet is very small
+            input1.joinWithTiny(input2)
+                  .where(0)
+                  .equalTo(0);
+
+DataSet<Tuple2<Tuple2<Integer, String>, Tuple2<Integer, String>>>
+            result2 =
+            // hint that the second DataSet is very large
+            input1.joinWithHuge(input2)
+                  .where(0)
+                  .equalTo(0);
+```
+
+### Cross
+
+The Cross transformation combines two `DataSet`s into one `DataSet`. It builds all pairwise combinations of the elements of both input `DataSet`s, i.e., it builds a Cartesian product.
+The Cross transformation either calls a user-defined `CrossFunction` on each pair of elements or applies a projection. Both modes are shown in the following.
+
+**Note:** Cross is potentially a *very* compute-intensive operation which can challenge even large compute clusters!
+
+#### Cross with User-Defined Function
+
+A Cross transformation can call a user-defined `CrossFunction`. A `CrossFunction` receives one element of the first input and one element of the second input and returns exactly one result element.
+
+The following code shows how to apply a Cross transformation on two `DataSet`s using a `CrossFunction`:
+
+```java
+public class Coord {
+  public int id;
+  public int x;
+  public int y;
+}
+
+// CrossFunction computes the Euclidean distance between two Coord objects.
+public class EuclideanDistComputer
+         extends CrossFunction<Coord, Coord, Tuple3<Integer, Integer, Double>> {
+
+  @Override
+  public Tuple3<Integer, Integer, Double> cross(Coord c1, Coord c2) {
+    // compute Euclidean distance of coordinates
+    double dist = Math.sqrt(Math.pow(c1.x - c2.x, 2) + Math.pow(c1.y - c2.y, 2));
+    return new Tuple3<Integer, Integer, Double>(c1.id, c2.id, dist);
+  }
+}
+
+DataSet<Coord> coords1 = // [...]
+DataSet<Coord> coords2 = // [...]
+DataSet<Tuple3<Integer, Integer, Double>>
+            distances =
+            coords1.cross(coords2)
+                   // apply CrossFunction
+                   .with(new EuclideanDistComputer());
+```
+
+#### Cross with Projection
+
+A Cross transformation can also construct result tuples using a projection as shown here:
+
+```java
+DataSet<Tuple3<Integer, Byte, String>> input1 = // [...]
+DataSet<Tuple2<Integer, Double>> input2 = // [...]
+DataSet<Tuple4<Integer, Byte, Integer, Double>
+            result =
+            input1.cross(input2)
+                  // select and reorder fields of matching tuples
+                  .projectSecond(0).projectFirst(1,0).projectSecond(1)
+                  .types(Integer.class, Byte.class, Integer.class, Double.class);
+```
+
+The field selection in a Cross projection works the same way as in the projection of Join results.
+
+#### Cross with DataSet Size Hint
+
+In order to guide the optimizer to pick the right execution strategy, you can hint the size of a `DataSet` to cross as shown here:
+
+```java
+DataSet<Tuple2<Integer, String>> input1 = // [...]
+DataSet<Tuple2<Integer, String>> input2 = // [...]
+
+DataSet<Tuple4<Integer, String, Integer, String>>
+            udfResult =
+                  // hint that the second DataSet is very small
+            input1.crossWithTiny(input2)
+                  // apply any Cross function (or projection)
+                  .with(new MyCrosser());
+
+DataSet<Tuple3<Integer, Integer, String>>
+            projectResult =
+                  // hint that the second DataSet is very large
+            input1.crossWithHuge(input2)
+                  // apply a projection (or any Cross function)
+                  .projectFirst(0,1).projectSecond(1).types(Integer.class, String.class, String.class)
+```
+
+### CoGroup
+
+The CoGroup transformation jointly processes groups of two `DataSet`s. Both `DataSet`s are grouped on a defined key and groups of both `DataSet`s that share the same key are handed together to a user-defined `CoGroupFunction`. If for a specific key only one `DataSet` has a group, the `CoGroupFunction` is called with this group and an empty group.
+A `CoGroupFunction` can separately iterate over the elements of both groups and return an arbitrary number of result elements.
+
+Similar to Reduce, GroupReduce, and Join, keys can be defined using
+
+- a `KeySelector` function or
+- one or more field position keys (`Tuple` `DataSet` only).
+
+#### CoGroup on DataSets grouped by Field Position Keys (Tuple DataSets only)
+
+```java
+// Some CoGroupFunction definition
+class MyCoGrouper
+         extends CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Double>, Double> {
+  // set to hold unique Integer values
+  Set<Integer> ints = new HashSet<Integer>();
+
+  @Override
+  public void coGroup(Iterator<Tuple2<String, Integer>> iVals,
+                      Iterator<Tuple2<String, Double>> dVals,
+                      Collector<Double> out) {
+    // clear Integer set
+    ints.clear();
+    // add all Integer values in group to set
+    while(iVals.hasNext()) {
+      ints.add(iVals.next().f1);
+    }
+    // multiply each Double value with each unique Integer values of group
+    while(dVals.hasNext()) {
+      for(Integer i : ints) {
+        out.collect(dVals.next().f1 * i));
+      }
+    }
+  }
+}
+
+// [...]
+DataSet<Tuple2<String, Integer>> iVals = // [...]
+DataSet<Tuple2<String, Double>> dVals = // [...]
+DataSet<Double> output = iVals.coGroup(dVals)
+                         // group first DataSet on first tuple field
+                         .where(0)
+                         // group second DataSet on first tuple field
+                         .equalTo(0)
+                         // apply CoGroup function on each pair of groups
+                         .with(new MyCoGrouper());
+```
+
+#### CoGroup on DataSets grouped by Key Selector Function
+
+Works analogous to key selector functions in Join transformations.
+
+### Union
+
+Produces the union of two `DataSet`s, which have to be of the same type. A union of more than two `DataSet`s can be implemented with multiple union calls, as shown here:
+
+```java
+DataSet<Tuple2<String, Integer>> vals1 = // [...]
+DataSet<Tuple2<String, Integer>> vals2 = // [...]
+DataSet<Tuple2<String, Integer>> vals3 = // [...]
+DataSet<Tuple2<String, Integer>> unioned = vals1.union(vals2)
+                    .union(vals3);
+```
+
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="data_sources">
+Data Sources
+------------
+
+Data sources create the initial data sets, such as from files or from Java collections. The general mechanism of of creating data sets is abstracted behind an [InputFormat](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/InputFormat.java). Stratosphere comes with several built-in formats to create data sets from common file formats. Many of them have shortcut methods on the *ExecutionEnvironment*.
+
+File-based:
+
+- `readTextFile(path)` / `TextInputFormat` - Reads files line wise and returns them as Strings.
+- `readTextFileWithValue(path)` / `TextValueInputFormat` - Reads files line wise and returns them as StringValues. StringValues are mutable strings.
+- `readCsvFile(path)` / `CsvInputFormat` - Parses files of comma (or another char) delimited fields. Returns a DataSet of tuples. Supports the basic java types and their Value counterparts as field types.
+
+Collection-based:
+
+- `fromCollection(Collection)` - Creates a data set from the Java Java.util.Collection. All elements in the collection must be of the same type.
+- `fromCollection(Iterator, Class)` - Creates a data set from an iterator. The class specifies the data type of the elements returned by the iterator.
+- `fromElements(T ...)` - Creates a data set from the given sequence of objects. All objects must be of the same type.
+- `fromParallelCollection(SplittableIterator, Class)` - Creates a data set from an iterator, in parallel. The class specifies the data type of the elements returned by the iterator.
+- `generateSequence(from, to)` - Generates the squence of numbers in the given interval, in parallel.
+
+Generic:
+
+- `createInput(path)` / `InputFormat` - Accepts a generic input format.
+
+**Examples**
+
+```java
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// read text file from local files system
+DataSet<String> localLines = env.readTextFile("file:///path/to/my/textfile");
+
+// read text file from a HDFS running at nnHost:nnPort
+DataSet<String> hdfsLines = env.readTextFile("hdfs://nnHost:nnPort/path/to/my/textfile");
+
+// read a CSV file with three fields
+DataSet<Tuple3<Integer, String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+	                       .types(Integer.class, String.class, Double.class);
+
+// read a CSV file with five fields, taking only two of them
+DataSet<Tuple2<String, Double>> csvInput = env.readCsvFile("hdfs:///the/CSV/file")
+                               .includeFields("10010")  // take the first and the fourth fild
+	                       .types(String.class, Double.class);
+
+// create a set from some given elements
+DataSet<String> value = env.fromElements("Foo", "bar", "foobar", "fubar");
+
+// generate a number sequence
+DataSet<Long> numbers = env.generateSequence(1, 10000000);
+
+// Read data from a relational database using the JDBC input format
+DataSet<Tuple2<String, Integer> dbData = 
+    env.createInput(
+      // create and configure input format
+      JDBCInputFormat.buildJDBCInputFormat()
+                     .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                     .setDBUrl("jdbc:derby:memory:persons")
+                     .setQuery("select name, age from persons")
+                     .finish(),
+      // specify type information for DataSet
+      new TupleTypeInfo(Tuple2.class, STRING_TYPE_INFO, INT_TYPE_INFO)
+    );
+
+// Note: Stratosphere's program compiler needs to infer the data types of the data items which are returned by an InputFormat. If this information cannot be automatically inferred, it is necessary to manually provide the type information as shown in the examples above.
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="data_sinks">
+Data Sinks
+----------
+
+Data sinks consume DataSets and are used to store or return them. Data sink operations are described using an [OutputFormat](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/OutputFormat.java). Stratosphere comes with a variety of built-in output formats that
+are encapsulated behind operations on the DataSet type:
+
+- `writeAsText()` / `TextOuputFormat` - Writes for each element as a String in a line. The String are obtained by calling the *toString()* method.
+- `writeAsCsv` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects.
+- `print()` / `printToErr()` - Prints the *toString()* value of each element on the standard out / strandard error stream.
+- `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports custom object-to-bytes conversion.
+- `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based (such as storing the result in a database).
+
+A DataSet can be input to multiple operations. Programs can write or print a data set and at the same time run additional transformations on them.
+
+**Examples**
+
+Standard data sink methods:
+
+```java
+// text data 
+DataSet<String> textData = // [...]
+
+// write DataSet to a file on the local file system
+textData.writeAsText("file:///my/result/on/localFS");
+
+// write DataSet to a file on a HDFS with a namenode running at nnHost:nnPort
+textData.writeAsText("hdfs://nnHost:nnPort/my/result/on/localFS");
+
+// write DataSet to a file and overwrite the file if it exists
+textData.writeAsText("file:///my/result/on/localFS", WriteMode.OVERWRITE);
+
+// tuples as lines with pipe as the separator "a|b|c"
+DataSet<Tuple3<String, Integer, Double>> values = // [...]
+values.writeAsCsv("file:///path/to/the/result/file", "\n", "|");
+
+// this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines
+value.writeAsText("file:///path/to/the/result/file");
+```
+
+Using a custom output format:
+
+```java
+DataSet<Tuple3<String, Integer, Double>> myResult = [...]
+
+// write Tuple DataSet to a relational database
+myResult.output(
+    // build and configure OutputFormat
+    JDBCOutputFormat.buildJDBCOutputFormat()
+                    .setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+                    .setDBUrl("jdbc:derby:memory:persons")
+                    .setQuery("insert into persons (name, age, height) values (?,?,?)")
+                    .finish()
+    );
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="debugging">
+Debugging
+---------
+
+Before running a data analysis program on a large data set in a distributed cluster, it is a good idea to make sure that the implemented algorithm works as desired. Hence, implementing data analysis programs is usually an incremental process of checking results, debugging, and improving. 
+
+<p>
+Stratosphere provides a few nice features to significantly ease the development process of data analysis programs by supporting local debugging from within an IDE, injection of test data, and collection of result data. This section give some hints how to ease the development of Stratosphere programs.
+</p>
+
+### Local Execution Environment
+
+A `LocalEnvironment` starts a Stratosphere system within the same JVM process it was created in. If you start the LocalEnvironement from an IDE, you can set breakpoint in your code and easily debug your program. 
+
+<p>
+A LocalEnvironment is created and used as follows:
+</p>
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+DataSet<String> lines = env.readTextFile(pathToTextFile);
+// build your program
+
+env.execute();
+
+```
+
+### Collection Data Sources and Sinks
+
+Providing input for an analysis program and checking its output is cumbersome done by creating input files and reading output files. Stratosphere features special data sources and sinks which are backed by Java collections to ease testing. Once a program has been tested, the sources and sinks can be easily replaced by sources and sinks that read from / write to external data stores such as HDFS.
+
+<p>
+Collection data sources can be used as follows:
+</p>
+
+```java
+final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+// Create a DataSet from a list of elements
+DataSet<Integer> myInts = env.fromElements(1, 2, 3, 4, 5);
+
+// Create a DataSet from any Java collection
+List<Tuple2<String, Integer>> data = ...
+DataSet<Tuple2<String, Integer>> myTuples = env.fromCollection(data);
+
+// Create a DataSet from an Iterator
+Iterator<Long> longIt = ...
+DataSet<Long> myLongs = env.fromCollection(longIt, Long.class);
+```
+
+**Note:** Currently, the collection data source requires that data types and iterators implement `Serializable`. Furthermore, collection data sources can not be executed in parallel (degree of parallelism = 1).
+
+<p>
+A collection data sink is specified as follows:
+</p>
+
+```java
+DataSet<Tuple2<String, Integer>> myResult = ...
+
+List<Tuple2<String, Integer>> outData = new ArrayList<Tuple2<String, Integer>>();
+myResult.output(new LocalCollectionOutputFormat(outData));
+```
+
+**Note:** Collection data sources will only work correctly, if the whole program is executed in the same JVM!
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="iterations">
+Iteration Operators
+-------------------
+
+Iterations implement loops in Stratosphere programs. The iteration operators encapsulate a part of the program and execute it repeatedly, feeding back the result of one iteration (the partial solution) into the next iteration. There are two types of iterations in Stratosphere: **BulkIteration** and **DeltaIteration**.
+
+This section provides quick examples on how to use both operators. Check out the [Introduction to Iterations]({{site.baseurl}}/docs/0.5/programming_guides/iterations.html) page for a more detailed introduction.
+
+#### Bulk Iterations
+
+To create a BulkIteration call the `iterate(int)` method of the `DataSet` the iteration should start at. This will return an `IterativeDataSet`, which can be transformed with the regular operators. The single argument to the iterate call specifies the maximum number of iterations.
+
+To specify the end of an iteration call the `closeWith(DataSet)` method on the `IterativeDataSet` to specify which transformation should be fed back to the next iteration. You can optionally specify a termination criterion with `closeWith(DataSet, DataSet)`, which evaluates the second DataSet and terminates the iteration, if this DataSet is empty. If no termination criterion is specified, the iteration terminates after the given maximum number iterations.
+
+The following example iteratively estimates the number Pi. The goal is to count the number of random points, which fall into the unit circle. In each iteration, a random point is picked. If this point lies inside the unit circle, we increment the count. Pi is then estimated as the resulting count divided by the number of iterations multiplied by 4.
+
+{% highlight java %}
+final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+// Create initial IterativeDataSet
+IterativeDataSet<Integer> initial = env.fromElements(0).iterate(10000);
+
+DataSet<Integer> iteration = initial.map(new MapFunction<Integer, Integer>() {
+    @Override
+    public Integer map(Integer i) throws Exception {
+        double x = Math.random();
+        double y = Math.random();
+
+        return i + ((x * x + y * y < 1) ? 1 : 0);
+    }
+});
+
+// Iteratively transform the IterativeDataSet
+DataSet<Integer> count = initial.closeWith(iteration);
+
+count.map(new MapFunction<Integer, Double>() {
+    @Override
+    public Double map(Integer count) throws Exception {
+        return count / (double) 10000 * 4;
+    }
+}).print();
+
+env.execute("Iterative Pi Example");
+{% endhighlight %}
+
+You can also check out the [K-Means example](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java), which uses a BulkIteration to cluster a set of unlabeled points.
+
+#### Delta Iterations
+
+Delta iterations exploit the fact that certain algorithms do not change every data point of the solution in each iteration.
+
+In addition to the partial solution that is fed back (called workset) in every iteration, delta iterations maintain state across iterations (called solution set), which can be updated through deltas. The result of the iterative computation is the state after the last iteration. Please refer to the [Introduction to Iterations]({{site.baseurl}}/docs/0.5/programming_guides/iterations.html) for an overview of the basic principle of delta iterations.
+
+Defining a DeltaIteration is similar to defining a BulkIteration. For delta iterations, two data sets form the input to each iteration (workset and solution set), and two data sets are produced as the result (new workset, solution set delta) in each iteration.
+
+To create a DeltaIteration call the `iterateDelta(DataSet, int, int)` (or `iterateDelta(DataSet, int, int[])` respectively). This method is called on the initial solution set. The arguments are the initial delta set, the maximum number of iterations and the key positions. The returned `DeltaIteration` object gives you access to the DataSets representing the workset and solution set via the methods `iteration.getWorket()` and `iteration.getSolutionSet()`.
+
+Below is an example for the syntax of a delta iteration
+
+```java
+// read the initial data sets
+DataSet<Tuple2<Long, Double>> initialSolutionSet = // [...]
+
+DataSet<Tuple2<Long, Double>> initialDeltaSet = // [...]
+
+int maxIterations = 100;
+int keyPosition = 0;
+
+DeltaIteration<Tuple2<Long, Double>, Tuple2<Long, Double>> iteration = initialSolutionSet
+    .iterateDelta(initialDeltaSet, maxIterations, keyPosition);
+
+DataSet<Tuple2<Long, Double>> candidateUpdates = iteration.getWorkset()
+    .groupBy(1)
+    .reduceGroup(new ComputeCandidateChanges());
+
+DataSet<Tuple2<Long, Double>> deltas = candidateUpdates
+    .join(iteration.getSolutionSet())
+    .where(0)
+    .equalTo(0)
+    .with(new CompareChangesToCurrent());
+
+DataSet<Tuple2<Long, Double>> nextWorkset = deltas
+    .filter(new FilterByThreshold());
+
+iteration.closeWith(deltas, nextWorkset)
+	.writeAsCsv(outputPath);
+```
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="annotations">
+Semantic Annotations
+-----------
+
+Semantic Annotations give hints about the behavior of a function by telling the system which fields in the input are accessed and which are constant between input and output data of a function (copied but not modified). Semantic annotations are a powerful means to speed up execution, because they allow the system to reason about reusing sort orders or partitions across multiple operations. Using semantic annotations may eventually save the program from unnecessary data shuffling or unnecessary sorts.
+
+Semantic annotations can be attached to functions through Java annotations, or passed as arguments when invoking a function on a DataSet. The following example illustrates that:
+
+```java
+@ConstantFields("1")
+public class DivideFirstbyTwo extends MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
+  @Override
+  public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
+    value.f0 /= 2;
+    return value;
+  }
+}
+```
+
+The following annotations are currently available:
+
+* `@ConstantFields`: Declares constant fields (forwarded/copied) for functions with a single input data set (Map, Reduce, Filter, ...).
+
+* `@ConstantFieldsFirst`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first input data set.
+
+* `@ConstantFieldsSecond`: Declares constant fields (forwarded/copied) for functions with a two input data sets (Join, CoGroup, ...), with respect to the first second data set.
+
+* `@ConstantFieldsExcept`: Declares that all fields are constant, except for the specified fields. Applicable to functions with a single input data set.
+
+* `@ConstantFieldsFirstExcept`: Declares that all fields of the first input are constant, except for the specified fields. Applicable to functions with a two input data sets.
+
+* `@ConstantFieldsSecondExcept`: Declares that all fields of the second input are constant, except for the specified fields. Applicable to functions with a two input data sets.
+
+*(Note: The system currently evaluated annotations only on Tuple data types. This will be extended in the next versions)*
+
+**Note**: It is important to be conservative when providing annotations. Only annotate fields, when they are always constant for every call to the function. Otherwise the system has incorrect assumptions about the execution and the execution may produce wrong results. If the behavior of the operator is not clearly predictable, no annotation should be provided.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="broadcast_variables">
+Broadcast Variables
+-------------------
+
+Broadcast variables allow you to make a data set available to all parallel instances of an operation, in addition to the regular input of the operation. This is useful
+for auxiliary data sets, or data-dependent parameterization. The data set will then be accessible at the operator as an `Collection<T>`.
+
+- **Broadcast**: broadcast sets are registered by name via `withBroadcastSet(DataSet, String)`, and
+- **Access**: accessible via `getRuntimeContext().getBroadcastVariable(String)` at the target operator.
+
+```java
+// 1. The DataSet to be broadcasted
+DataSet<Integer> toBroadcast = env.fromElements(1, 2, 3);
+
+DataSet<String> data = env.fromElements("a", "b");
+
+data.map(new MapFunction<String, String>() {
+    @Override
+    public void open(Configuration parameters) throws Exception {
+      // 3. Access the broadcasted DataSet as a Collection
+      Collection<Integer> broadcastSet = getRuntimeContext().getBroadcastVariable("broadcastSetName");
+    }
+
+
+    @Override
+    public String map(String value) throws Exception {
+        ...
+    }
+}).withBroadcastSet(toBroadcast, "broadcastSetName"); // 2. Broadcast the DataSet
+```
+
+Make sure that the names (`broadcastSetName` in the previous example) match when registering and accessing broadcasted data sets. For a complete example program, have a look at [KMeans Algorithm](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/clustering/KMeans.java#L96).
+
+**Note**: As the content of broadcast variables is kept in-memory on each node, it should not become too large. For simpler things like scalar values you can simply make parameters part of the closure of a function, or use the `withParameters(...)` method to pass in a configuration.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="packaging">
+Program Packaging & Distributed Execution
+-----------------------------------------
+
+As described in the [program skeleton](#skeleton) section, Stratosphere programs can be executed on clusters by using the `RemoteEnvironment`. Alternatively, programs can be packaged into JAR Files (Java Archives) for execution. Packaging the program is a prerequisite to executing them through the [command line interface]({{ site.baseurl }}/docs/0.5/program_execution/cli_client.html) or the [web interface]({{ site.baseurl }}/docs/0.5/program_execution/web_interface.html).
+
+#### Packaging Programs
+
+To support execution from a packaged JAR file via the command line or web interface, a program must use the environment obtained by `ExecutionEnvironment.getExecutionEnvironment()`. This environment will act as the cluster's environment when the JAR is submitted to the command line or web interface. If the Stratosphere program is invoked differently than through these interfaces, the environment will act like a local environment.
+
+To package the program, simply export all involved classes as a JAR file. The JAR file's manifest must point to the class that contains the program's *entry point* (the class with the `public void main(String[])` method). The simplest way to do this is by putting the *main-class* entry into the manifest (such as `main-class: eu.stratosphere.example.MyProgram`). The *main-class* attribute is the same one that is used by the Java Virtual Machine to find the main method when executing a JAR files through the command `java -jar pathToTheJarFile`. Most IDEs offer to include that attribute automatically when exporting JAR files.
+
+
+#### Packaging Programs through Plans
+
+Additionally, the Java API supports packaging programs as *Plans*. This method resembles the way that the *Scala API* packages programs. Instead of defining a progam in the main method and calling `execute()` on the environment, plan packaging returns the *Program Plan*, which is a description of the program's data flow. To do that, the program must implement the `eu.stratosphere.api.common.Program` interface, defining the `getPlan(String...)` method. The strings passed to that method are the command line arguments. The program's plan can be created from the environment via the `ExecutionEnvironment#createProgramPlan()` method. When packaging the program's plan, the JAR manifest must point to the class implementing the `eu.stratosphere.api.common.Program` interface, instead of the class with the main method.
+
+
+#### Summary
+
+The overall procedure to invoke a packaged program is as follows:
+
+  1. The JAR's manifest is searched for a *main-class* or *program-class* attribute. If both attributes are found, the *program-class* attribute takes precedence over the *main-class* attribute. Both the command line and the web interface support a parameter to pass the entry point class name manually for cases where the JAR manifest contains neither attribute.
+  2. If the entry point class implements the `eu.stratosphere.api.common.Program`, then the system calls the `getPlan(String...)` method to obtain the program plan to execute. The `getPlan(String...)` method was the only possible way of defining a program in the *Record API* (see [0.4 docs]({{ site.baseurl }}/docs/0.4/)) and is also supported in the new Java API.
+  3. If the entry point class does not implement the `eu.stratosphere.api.common.Program` interface, the system will invoke the main method of the class.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="accumulators_counters">
+Accumulators &amp; Counters
+---------------------------
+
+Accumulators are simple constructs with an **add operation** and a **final accumulated result**, which is available after the job ended.
+
+The most straightforward accumulator is a **counter**: You can increment it using the ```Accumulator.add(V value)``` method. At the end of the job Stratosphere will sum up (merge) all partial results and send the result to the client. Since accumulators are very easy to use, they can be useful during debugging or if you quickly want to find out more about your data.
+
+Stratosphere currently has the following **built-in accumulators**. Each of them implements the [Accumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Accumulator.java) interface.
+
+- [__IntCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java), [__LongCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java) and [__DoubleCounter__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java): See below for an example using a counter.
+- [__Histogram__](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java): A histogram implementation for a discrete number of bins. Internally it is just a map from Integer to Integer. You can use this to compute distributions of values, e.g. the distribution of words-per-line for a word count program.
+
+__How to use accumulators:__
+
+First you have to create an accumulator object (here a counter) in the operator function where you want to use it. Operator function here refers to the (anonymous inner)
+class implementing the user defined code for an operator.
+
+    private IntCounter numLines = new IntCounter();
+
+Second you have to register the accumulator object, typically in the ```open()``` method of the operator function. Here you also define the name.
+
+    getRuntimeContext().addAccumulator("num-lines", this.numLines);
+
+You can now use the accumulator anywhere in the operator function, including in the ```open()``` and ```close()``` methods.
+
+    this.numLines.add(1);
+
+The overall result will be stored in the ```JobExecutionResult``` object which is returned when running a job using the Java API (currently this only works if the execution waits for the completion of the job).
+
+    myJobExecutionResult.getAccumulatorResult("num-lines")
+
+All accumulators share a single namespace per job. Thus you can use the same accumulator in different operator functions of your job. Stratosphere will internally merge all accumulators with the same name.
+
+A note on accumulators and iterations: Currently the result of accumulators is only available after the overall job ended. We plan to also make the result of the previous iteration available in the next iteration. You can use [Aggregators](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/IterativeDataSet.java#L98) to compute per-iteration statistics and base the termination of iterations on such statistics.
+
+__Custom accumulators:__
+
+To implement your own accumulator you simply have to write your implementation of the Accumulator interface. Feel free to create a pull request if you think your custom accumulator should be shipped with Stratosphere.
+
+You have the choice to implement either [Accumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Accumulator.java) or [SimpleAccumulator](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/SimpleAccumulator.java). ```Accumulator<V,R>``` is most flexible: It defines a type ```V``` for the value to add, and a result type ```R``` for the final result. E.g. for a histogram, ```V``` is a number and ```R``` is a histogram. ```SimpleAccumulator``` is for the cases where both types are the same, e.g. for counters.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>
+
+<section id="execution_plan">
+Execution Plans
+---------------
+
+Depending on various parameters such as data size or number of machines in the cluster, Stratosphere's optimizer automatically chooses an execution strategy for your program. In many cases, it can be useful to know how exactly Stratosphere will execute your program.
+
+__Plan Visualization Tool__
+
+Stratosphere 0.5 comes packaged with a visualization tool for execution plans. The HTML document containing the visualizer is located under ```tools/planVisualizer.html```. It takes a JSON representation of the job execution plan and visualizes it as a graph with complete annotations of execution strategies.
+
+The following code shows how to print the execution plan JSON from your program:
+
+    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    ...
+
+    System.out.println(env.getExecutionPlan());
+
+
+To visualize the execution plan, do the following:
+
+1. **Open** ```planVisualizer.html``` with your web browser,
+2. **Paste** the JSON string into the text field, and
+3. **Press** the draw button.
+
+After these steps, a detailed execution plan will be visualized.
+
+![alt text](http://stratosphere.eu/img/blog/plan_visualizer2.png "A stratosphere job execution graph.")
+
+__Web Interface__
+
+Stratosphere offers a web interface for submitting and executing jobs. If you choose to use this interface to submit your packaged program, you have the option to also see the plan visualization.
+
+The script to start the webinterface is located under ```bin/start-webclient.sh```. After starting the webclient (per default on **port 8080**), your program can be uploaded and will be added to the list of available programs on the left side of the interface.
+
+You are able to specify program arguments in the textbox at the bottom of the page. Checking the plan visualization checkbox shows the execution plan before executing the actual program.
+
+<div class="back-to-top"><a href="#toc">Back to top</a></div>
+</section>


[7/7] git commit: [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
[FLINK-962] Initial import of documentation from website into source code (closes #34)


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/40b94f73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/40b94f73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/40b94f73

Branch: refs/heads/master
Commit: 40b94f73300788e191d32c5918e647bff748e151
Parents: b4b633e
Author: uce <u....@fu-berlin.de>
Authored: Sat Jun 21 15:14:05 2014 +0200
Committer: uce <u....@fu-berlin.de>
Committed: Mon Jun 23 14:51:07 2014 +0200

----------------------------------------------------------------------
 docs/README.md                                  |   60 +
 docs/_config.yml                                |   33 +
 docs/_layouts/docs.html                         |   92 ++
 docs/_plugins/tocify.rb                         |   10 +
 docs/build_docs.sh                              |   58 +
 docs/cli.md                                     |  129 ++
 docs/cluster_execution.md                       |  125 ++
 docs/cluster_setup.md                           |  363 +++++
 docs/config.md                                  |  171 ++
 docs/css/syntax.css                             |   60 +
 docs/faq.md                                     |  285 ++++
 docs/hadoop_compatability.md                    |    5 +
 docs/img/cogroup.svg                            |  856 ++++++++++
 docs/img/cross.svg                              |  893 +++++++++++
 docs/img/dataflow.svg                           |  979 ++++++++++++
 docs/img/datatypes.svg                          |  143 ++
 docs/img/iterations_delta_iterate_operator.png  |  Bin 0 -> 113607 bytes
 ...terations_delta_iterate_operator_example.png |  Bin 0 -> 335057 bytes
 docs/img/iterations_iterate_operator.png        |  Bin 0 -> 63465 bytes
 .../img/iterations_iterate_operator_example.png |  Bin 0 -> 102925 bytes
 docs/img/iterations_supersteps.png              |  Bin 0 -> 54098 bytes
 docs/img/japi_example_overview.png              |  Bin 0 -> 45406 bytes
 docs/img/join.svg                               |  615 ++++++++
 docs/img/map.svg                                |  295 ++++
 docs/img/operator.svg                           |  241 +++
 docs/img/recorddm.svg                           |  263 ++++
 docs/img/reduce.svg                             |  425 +++++
 docs/img/spargel_example.png                    |  Bin 0 -> 199032 bytes
 docs/img/spargel_example_input.png              |  Bin 0 -> 113478 bytes
 docs/index.md                                   |   11 +
 docs/iterations.md                              |  188 +++
 docs/java_api_examples.md                       |  304 ++++
 docs/java_api_guide.md                          | 1476 ++++++++++++++++++
 docs/java_api_quickstart.md                     |  126 ++
 docs/local_execution.md                         |  106 ++
 docs/local_setup.md                             |  108 ++
 docs/quickstart/plotPoints.py                   |   82 +
 docs/run_example_quickstart.md                  |  154 ++
 docs/scala_api_examples.md                      |  195 +++
 docs/scala_api_guide.md                         | 1008 ++++++++++++
 docs/scala_api_quickstart.md                    |   71 +
 docs/setup_quickstart.md                        |  132 ++
 docs/spargel_guide.md                           |  112 ++
 docs/web_client.md                              |   53 +
 docs/yarn_setup.md                              |  188 +++
 pom.xml                                         |    2 +
 46 files changed, 10417 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
new file mode 100644
index 0000000..4ecb30e
--- /dev/null
+++ b/docs/README.md
@@ -0,0 +1,60 @@
+This README gives an overview of how to build and contribute to the
+documentation of Apache Flink.
+
+The documentation is included with the source of Apache Flink in order to ensure
+that you always have docs corresponding to your checked out version. The online
+documentation at http://flink.incubator.apache.org/ is also generated from the
+files found here.
+
+# Requirements
+
+We use Markdown to write and Jekyll to translate the documentation to static
+HTML. You can install all needed software via:
+
+    gem install jekyll
+    gem install redcarpet
+    sudo easy_install Pygments
+
+Redcarpet is needed for Markdown processing and the Python based Pygments is
+used for syntax highlighting.
+
+# Build
+
+The `docs/build_docs.sh` script calls Jekyll and generates the documentation to
+`docs/target`. You can then point your browser to `docs/target/index.html` and
+start reading.
+
+If you call the script with the preview flag `build_docs.sh -p`, Jekyll will
+start a web server at `localhost:4000` and continiously generate the docs.
+This is useful to preview changes locally.
+
+# Contribute
+
+The documentation pages are written in
+[Markdown](http://daringfireball.net/projects/markdown/syntax). It is possible
+to use the [GitHub flavored syntax](http://github.github.com/github-flavored-markdown)
+and intermix plain html.
+
+In addition to Markdown, every page contains a front matter, which specifies the
+title of the page. This title is used as the top-level heading for the page.
+
+    ---
+    title: "Title of the Page"
+    ---
+
+Furthermore, you can access variables found in `docs/_config.yml` as follows:
+
+    {{ site.FLINK_VERSION }}
+
+This will be replaced with the value of the variable when generating the docs.
+
+All documents are structed with headings. From these heading, an page outline is
+automatically generated for each page.
+
+```
+# Level-1 Heading
+## Level-2 Heading
+### Level-3 heading
+#### Level-4 heading
+##### Level-5 heading
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
new file mode 100644
index 0000000..2d7c1ce
--- /dev/null
+++ b/docs/_config.yml
@@ -0,0 +1,33 @@
+#------------------------------------------------------------------------------
+# VARIABLES
+#------------------------------------------------------------------------------
+# Variables specified in this file can be used in the documentation via:
+#     {{ site.CONFIG_KEY }}
+#------------------------------------------------------------------------------
+
+FLINK_VERSION: 0.6-SNAPSHOT
+FLINK_VERSION_SHORT: 0.6
+FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK
+FLINK_GITHUB_URL:  https://github.com/apache/incubator-flink
+
+#------------------------------------------------------------------------------
+# BUILD CONFIG
+#------------------------------------------------------------------------------
+# These variables configure the jekyll build (./build_docs.sh). You don't need
+# to change anything here.
+#------------------------------------------------------------------------------
+
+defaults:
+  -
+    scope:
+      path: ""
+    values:
+      layout: docs
+
+highlighter: pygments
+markdown: redcarpet
+redcarpet:
+  # https://help.github.com/articles/github-flavored-markdown
+  extensions: ["no_intra_emphasis", "fenced_code_blocks", "autolink",
+               "tables", "with_toc_data", "strikethrough", "superscript",
+               "lax_spacing"]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/docs.html b/docs/_layouts/docs.html
new file mode 100644
index 0000000..4b99d4a
--- /dev/null
+++ b/docs/_layouts/docs.html
@@ -0,0 +1,92 @@
+<!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Apache Flink {{ site.FLINK_VERSION }} Documentation: {{ page.title }}</title>
+
+    <link rel="stylesheet" href="//netdna.bootstrapcdn.com/bootstrap/3.1.1/css/bootstrap.min.css">
+    <link rel="stylesheet" href="css/syntax.css">
+
+    <!-- HTML5 Shim and Respond.js IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+      <script src="https://oss.maxcdn.com/libs/html5shiv/3.7.0/html5shiv.js"></script>
+      <script src="https://oss.maxcdn.com/libs/respond.js/1.4.2/respond.min.js"></script>
+    <![endif]-->
+  </head>
+  <body>
+    <div class="container">
+        <div class="row">
+            <h1>Apache Flink {{ site.FLINK_VERSION }} Documentation</h1>
+        </div>
+        <div class="row">
+            <div class="col-md-3">
+                <ul>
+                    <li>Quickstart
+                        <ul>
+                            <li><a href="setup_quickstart.html">Install</a></li>
+                            <li><a href="run_example_quickstart.html">Run Example</a></li>
+                            <li><a href="java_api_quickstart.html">Java API</a></li>
+                            <li><a href="scala_api_quickstart.html">Scala API</a></li>
+                            <li><a href="faq.html">FAQ</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Setup &amp; Configuration
+                        <ul>
+                            <li><a href="local_setup.html">Local Setup</a></li>
+                            <li><a href="cluster_setup.html">Cluster Setup</a></li>
+                            <li><a href="yarn_setup.html">YARN Setup</a></li>
+                            <li><a href="config.html">Configuration</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Programming Guides
+                        <ul>
+                            <li><a href="java_api_guide.html">Java API</a></li>
+                            <li><a href="scala_api_guide.html">Scala API</a></li>
+                            <li><a href="hadoop_compatability.html">Hadoop Compatability</a></li>
+                            <li><a href="iterations.html">Iterations</a></li>
+                            <li><a href="spargel_guide.html">Spargel Graph API</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Examples
+                        <ul>
+                            <li><a href="java_api_examples.html">Java API</a></li>
+                            <li><a href="scala_api_examples.html">Scala API</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Execution
+                        <ul>
+                            <li><a href="local_execution.html">Local/Debugging</a></li>
+                            <li><a href="cluster_execution.html">Cluster</a></li>
+                            <li><a href="cli.html">Command-Line Interface</a></li>
+                            <li><a href="web_client.html">Web Interface</a></li>
+                        </ul>
+                    </li>
+
+                    <li>Internals
+                        <ul>
+                            <li>To be written</li>
+                        </ul>
+                    </li>
+                </ul>
+            </div>
+            <div class="col-md-9">
+                <h1>{{ page.title }}</h1>
+
+                {{ page.content | tocify }}
+
+                {{ content }}
+            </div>
+        </div>
+    </div>
+
+    <script src="https://ajax.googleapis.com/ajax/libs/jquery/1.11.0/jquery.min.js"></script>
+    <script src="//netdna.bootstrapcdn.com/bootstrap/3.1.1/js/bootstrap.min.js"></script>
+  </body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/_plugins/tocify.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/tocify.rb b/docs/_plugins/tocify.rb
new file mode 100644
index 0000000..7df0c3d
--- /dev/null
+++ b/docs/_plugins/tocify.rb
@@ -0,0 +1,10 @@
+module Jekyll
+  module Tocify
+    def tocify(input)
+      converter = Redcarpet::Markdown.new(Redcarpet::Render::HTML_TOC)
+      converter.render(input)
+    end
+  end
+end
+
+Liquid::Template.register_filter(Jekyll::Tocify)

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/build_docs.sh
----------------------------------------------------------------------
diff --git a/docs/build_docs.sh b/docs/build_docs.sh
new file mode 100755
index 0000000..7ae3343
--- /dev/null
+++ b/docs/build_docs.sh
@@ -0,0 +1,58 @@
+#!/bin/bash
+########################################################################################################################
+# Copyright (C) 2010-2014 by the Stratos	phere project (http://stratosphere.eu)
+#
+# Licensed 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.
+########################################################################################################################
+
+HAS_JEKYLL=true
+
+command -v jekyll > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "ERROR: Could not find jekyll. "
+	echo "Please install with 'gem install jekyll' (see http://jekyllrb.com)."
+
+	HAS_JEKYLL=false
+fi
+
+command -v redcarpet > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "WARN: Could not find redcarpet. "
+	echo -n "Please install with 'sudo gem install redcarpet' (see https://github.com/vmg/redcarpet). "
+	echo "Redcarpet is needed for Markdown parsing and table of contents generation."
+fi
+
+command -v pygmentize > /dev/null
+if [ $? -ne 0 ]; then
+	echo -n "WARN: Could not find pygments. "
+	echo -n "Please install with 'sudo easy_install Pygments' (requires Python; see http://pygments.org). "
+	echo "Pygments is needed for syntax highlighting of the code examples."
+fi
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+
+DOCS_SRC=${DIR}
+DOCS_DST=${DOCS_SRC}/target
+
+# default jekyll command is to just build site
+JEKYLL_CMD="build"
+
+# if -p flag is provided, serve site on localhost
+while getopts ":p" opt; do
+	case $opt in
+		p)
+		JEKYLL_CMD="serve --watch"
+		;;
+	esac
+done
+
+if $HAS_JEKYLL; then
+	jekyll ${JEKYLL_CMD} --source ${DOCS_SRC} --destination ${DOCS_DST}
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cli.md
----------------------------------------------------------------------
diff --git a/docs/cli.md b/docs/cli.md
new file mode 100644
index 0000000..0e778da
--- /dev/null
+++ b/docs/cli.md
@@ -0,0 +1,129 @@
+---
+title:  "Command-Line Interface"
+---
+
+Stratosphere provides a command-line interface to run programs that are packaged
+as JAR files, and control their execution.  The command line interface is part
+of any Stratosphere setup, available in local single node setups and in
+distributed setups. It is located under `<stratosphere-home>/bin/stratosphere`
+and connects by default to the running Stratosphere master (JobManager) that was
+started from the same installation directory.
+
+A prerequisite to using the command line interface is that the Stratosphere
+master (JobManager) has been started (via `<stratosphere-home>/bin/start-
+local.sh` or `<stratosphere-home>/bin/start-cluster.sh`).
+
+The command line can be used to
+
+- submit jobs for execution,
+- cancel a running job,
+- provide information about a job, and
+- list running and waiting jobs.
+
+# Examples
+
+-   Run example program with no arguments.
+
+        ./bin/stratosphere run ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar
+
+-   Run example program with arguments for input and result files
+
+        ./bin/stratosphere run ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                               file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   Run example program with parallelism 16 and arguments for input and result files
+
+        ./bin/stratosphere run -p 16 ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                                file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   Run example program on a specific JobManager:
+
+        ./bin/stratosphere run -m myJMHost:6123 \
+                               ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                               -file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+
+-   Display the expected arguments for the WordCount example program:
+
+        ./bin/stratosphere info -d ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar
+
+-   Display the optimized execution plan for the WordCount example program as JSON:
+
+        ./bin/stratosphere info -e 
+                                ./examples/stratosphere-java-examples-{{ site.FLINK_VERSION }}-WordCount.jar \
+                                file:///home/user/hamlet.txt file:///home/user/wordcount_out
+
+-   List scheduled and running jobs (including their JobIDs):
+
+        ./bin/stratosphere list -s -r
+
+-   Cancel a job:
+
+        ./bin/stratosphere cancel -i <jobID>
+
+# Usage
+
+The command line syntax is as follows:
+
+```
+./stratosphere <ACTION> [OPTIONS] [ARGUMENTS]
+
+General options:
+     -h,--help      Show the help for the CLI Frontend, or a specific action.
+     -v,--verbose   Print more detailed error messages.
+
+
+Action "run" - compiles and submits a Stratosphere program that is given in the form of a JAR file.
+
+  "run" options:
+
+     -p,--parallelism <parallelism> The degree of parallelism for the execution. This value is used unless the program overrides the degree of parallelism on the execution environment or program plan. If this option is not set, then the execution will use the default parallelism specified in the stratosphere-conf.yaml file.
+
+     -c,--class <classname>         The class with the entry point (main method, or getPlan() method). Needs only be specified if the JAR file has no manifest pointing to that class. See program packaging instructions for details.
+
+     -m,--jobmanager <host:port>    Option to submit the program to a different Stratosphere master (JobManager).
+
+  "run" arguments:
+
+     - The first argument is the path to the JAR file of the program.
+     - All successive arguments are passed to the program's main method (or getPlan() method).
+
+
+Action "info" - displays information about a Stratosphere program.
+
+  "info" action arguments:
+     -d,--description               Show description of the program, if the main class implements the 'ProgramDescription' interface.
+
+     -e,--executionplan             Show the execution data flow plan of the program, in JSON representation.
+
+     -p,--parallelism <parallelism> The degree of parallelism for the execution, see above. The parallelism is relevant for the execution plan. The option is only evaluated if used together with the -e option.
+
+     -c,--class <classname>         The class with the entry point (main method, or getPlan() method). Needs only be specified if the JAR file has no manifest pointing to that class. See program packaging instructions for details.
+
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager). Connecting to a master is relevant to compile the execution plan. The option is only evaluated if used together with the -e option.
+
+  "info" arguments:
+
+     - The first argument is the path to the JAR file of the program.
+     - All successive arguments are passed to the program's main method (or getPlan() method).
+
+
+Action "list" lists submitted Stratosphere programs.
+
+  "list" action arguments:
+
+     -r,--running                   Show running programs and their JobIDs
+
+     -s,--scheduled                 Show scheduled programs and their JobIDs
+
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager).
+
+
+Action "cancel" cancels a submitted Stratosphere program.
+
+  "cancel" action arguments:
+
+     -i,--jobid <jobID>             JobID of program to cancel
+     
+     -m,--jobmanager <host:port>    Option to connect to a different Stratosphere master (JobManager).
+```

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cluster_execution.md
----------------------------------------------------------------------
diff --git a/docs/cluster_execution.md b/docs/cluster_execution.md
new file mode 100644
index 0000000..a41bc0f
--- /dev/null
+++ b/docs/cluster_execution.md
@@ -0,0 +1,125 @@
+---
+title:  "Cluster Execution"
+---
+
+Stratosphere programs can run distributed on clusters of many machines. There
+are two ways to send a program to a cluster for execution:
+
+# Command Line Interface
+
+The command line interface lets you submit packaged programs (JARs) to a cluster
+(or single machine setup).
+
+Please refer to the [Command Line Interface](cli.html) documentation for
+details.
+
+# Remote Environment
+
+The remote environment lets you execute Stratosphere Java programs on a cluster
+directly. The remote environment points to the cluster on which you want to
+execute the program.
+
+## Maven Dependency
+
+If you are developing your program as a Maven project, you have to add the
+`stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{ site.FLINK_VERSION }}</version>
+</dependency>
+```
+
+## Example
+
+The following illustrates the use of the `RemoteEnvironment`:
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment
+        .createRemoteEnvironment("strato-master", "7661", "/home/user/udfs.jar");
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    env.execute();
+}
+```
+
+Note that the program contains custom UDFs and hence requires a JAR file with
+the classes of the code attached. The constructor of the remote environment
+takes the path(s) to the JAR file(s).
+
+# Remote Executor
+
+Similar to the RemoteEnvironment, the RemoteExecutor lets you execute
+Stratosphere programs on a cluster directly. The remote executor accepts a
+*Plan* object, which describes the program as a single executable unit.
+
+## Maven Dependency
+
+If you are developing your program in a Maven project, you have to add the
+`stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{ site.FLINK_VERSION }}</version>
+</dependency>
+```
+
+## Example
+
+The following illustrates the use of the `RemoteExecutor` with the Scala API:
+
+```scala
+def main(args: Array[String]) {
+    val input = TextFile("hdfs://path/to/file")
+
+    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
+    val counts = words groupBy { x => x } count()
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+  
+    val plan = new ScalaPlan(Seq(output), "Word Count")
+    val executor = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar")
+    executor.executePlan(p);
+}
+```
+
+The following illustrates the use of the `RemoteExecutor` with the Java API (as
+an alternative to the RemoteEnvironment):
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    Plan p = env.createProgramPlan();
+    RemoteExecutor e = new RemoteExecutor("strato-master", 7881, "/path/to/jarfile.jar");
+    e.executePlan(p);
+}
+```
+
+Note that the program contains custom UDFs and hence requires a JAR file with
+the classes of the code attached. The constructor of the remote executor takes
+the path(s) to the JAR file(s).

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/cluster_setup.md
----------------------------------------------------------------------
diff --git a/docs/cluster_setup.md b/docs/cluster_setup.md
new file mode 100644
index 0000000..3692882
--- /dev/null
+++ b/docs/cluster_setup.md
@@ -0,0 +1,363 @@
+---
+title:  "Cluster Setup"
+---
+
+This documentation is intended to provide instructions on how to run
+Stratosphere in a fully distributed fashion on a static (but possibly
+heterogeneous) cluster.
+
+This involves two steps. First, installing and configuring Stratosphere and
+second installing and configuring the [Hadoop Distributed
+Filesystem](http://hadoop.apache.org/) (HDFS).
+
+# Preparing the Cluster
+
+## Software Requirements
+
+Stratosphere runs on all *UNIX-like environments*, e.g. **Linux**, **Mac OS X**,
+and **Cygwin** (for Windows) and expects the cluster to consist of **one master
+node** and **one or more worker nodes**. Before you start to setup the system,
+make sure you have the following software installed **on each node**:
+
+- **Java 1.6.x** or higher,
+- **ssh** (sshd must be running to use the Stratosphere scripts that manage
+  remote components)
+
+If your cluster does not fulfill these software requirements you will need to
+install/upgrade it.
+
+For example, on Ubuntu Linux, type in the following commands to install Java and
+ssh:
+
+```
+sudo apt-get install ssh 
+sudo apt-get install openjdk-7-jre
+```
+
+You can check the correct installation of Java by issuing the following command:
+
+```
+java -version
+```
+
+The command should output something comparable to the following on every node of
+your cluster (depending on your Java version, there may be small differences):
+
+```
+java version "1.6.0_22"
+Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
+Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
+```
+
+To make sure the ssh daemon is running properly, you can use the command
+
+```
+ps aux | grep sshd
+```
+
+Something comparable to the following line should appear in the output
+of the command on every host of your cluster:
+
+```
+root       894  0.0  0.0  49260   320 ?        Ss   Jan09   0:13 /usr/sbin/sshd
+```
+
+## Configuring Remote Access with ssh
+
+In order to start/stop the remote processes, the master node requires access via
+ssh to the worker nodes. It is most convenient to use ssh's public key
+authentication for this. To setup public key authentication, log on to the
+master as the user who will later execute all the Stratosphere components. **The
+same user (i.e. a user with the same user name) must also exist on all worker
+nodes**. For the remainder of this instruction we will refer to this user as
+*stratosphere*. Using the super user *root* is highly discouraged for security
+reasons.
+
+Once you logged in to the master node as the desired user, you must generate a
+new public/private key pair. The following command will create a new
+public/private key pair into the *.ssh* directory inside the home directory of
+the user *stratosphere*. See the ssh-keygen man page for more details. Note that
+the private key is not protected by a passphrase.
+
+```
+ssh-keygen -b 2048 -P '' -f ~/.ssh/id_rsa
+```
+
+Next, copy/append the content of the file *.ssh/id_rsa.pub* to your
+authorized_keys file. The content of the authorized_keys file defines which
+public keys are considered trustworthy during the public key authentication
+process. On most systems the appropriate command is
+
+```
+cat .ssh/id_rsa.pub >> .ssh/authorized_keys
+```
+
+On some Linux systems, the authorized keys file may also be expected by the ssh
+daemon under *.ssh/authorized_keys2*. In either case, you should make sure the
+file only contains those public keys which you consider trustworthy for each
+node of cluster.
+
+Finally, the authorized keys file must be copied to every worker node of your
+cluster. You can do this by repeatedly typing in
+
+```
+scp .ssh/authorized_keys <worker>:~/.ssh/
+```
+
+and replacing *\<worker\>* with the host name of the respective worker node.
+After having finished the copy process, you should be able to log on to each
+worker node from your master node via ssh without a password.
+
+## Setting JAVA_HOME on each Node
+
+Stratosphere requires the `JAVA_HOME` environment variable to be set on the
+master and all worker nodes and point to the directory of your Java
+installation.
+
+You can set this variable in `conf/stratosphere-conf.yaml` via the
+`env.java.home` key.
+
+Alternatively, add the following line to your shell profile. If you use the
+*bash* shell (probably the most common shell), the shell profile is located in
+*\~/.bashrc*:
+
+```
+export JAVA_HOME=/path/to/java_home/
+```
+
+If your ssh daemon supports user environments, you can also add `JAVA_HOME` to
+*.\~/.ssh/environment*. As super user *root* you can enable ssh user
+environments with the following commands:
+
+```
+echo "PermitUserEnvironment yes" >> /etc/ssh/sshd_config
+/etc/init.d/ssh restart
+```
+
+# Hadoop Distributed Filesystem (HDFS) Setup
+
+The Stratosphere system currently uses the Hadoop Distributed Filesystem (HDFS)
+to read and write data in a distributed fashion.
+
+Make sure to have a running HDFS installation. The following instructions are
+just a general overview of some required settings. Please consult one of the
+many installation guides available online for more detailed instructions.
+
+**Note that the following instructions are based on Hadoop 1.2 and might differ
+**for Hadoop 2.
+
+## Downloading, Installing, and Configuring HDFS
+
+Similar to the Stratosphere system HDFS runs in a distributed fashion. HDFS
+consists of a **NameNode** which manages the distributed file system's meta
+data. The actual data is stored by one or more **DataNodes**. For the remainder
+of this instruction we assume the HDFS's NameNode component runs on the master
+node while all the worker nodes run an HDFS DataNode.
+
+To start, log on to your master node and download Hadoop (which includes  HDFS)
+from the Apache [Hadoop Releases](http://hadoop.apache.org/releases.html) page.
+
+Next, extract the Hadoop archive.
+
+After having extracted the Hadoop archive, change into the Hadoop directory and
+edit the Hadoop environment configuration file:
+
+```
+cd hadoop-*
+vi conf/hadoop-env.sh
+```
+
+Uncomment and modify the following line in the file according to the path of
+your Java installation.
+
+``` export JAVA_HOME=/path/to/java_home/ ```
+
+Save the changes and open the HDFS configuration file *conf/hdfs-site.xml*. HDFS
+offers multiple configuration parameters which affect the behavior of the
+distributed file system in various ways. The following excerpt shows a minimal
+configuration which is required to make HDFS work. More information on how to
+configure HDFS can be found in the [HDFS User
+Guide](http://hadoop.apache.org/docs/r1.2.1/hdfs_user_guide.html) guide.
+
+```xml
+<configuration>
+  <property>
+    <name>fs.default.name</name>
+    <value>hdfs://MASTER:50040/</value>
+  </property>
+  <property>
+    <name>dfs.data.dir</name>
+    <value>DATAPATH</value>
+  </property>
+</configuration>
+```
+
+Replace *MASTER* with the IP/host name of your master node which runs the
+*NameNode*. *DATAPATH* must be replaced with path to the directory in which the
+actual HDFS data shall be stored on each worker node. Make sure that the
+*stratosphere* user has sufficient permissions to read and write in that
+directory.
+
+After having saved the HDFS configuration file, open the file *conf/slaves* and
+enter the IP/host name of those worker nodes which shall act as *DataNode*s.
+Each entry must be separated by a line break.
+
+```
+<worker 1>
+<worker 2>
+.
+.
+.
+<worker n>
+```
+
+Initialize the HDFS by typing in the following command. Note that the
+command will **delete all data** which has been previously stored in the
+HDFS. However, since we have just installed a fresh HDFS, it should be
+safe to answer the confirmation with *yes*.
+
+```
+bin/hadoop namenode -format
+```
+
+Finally, we need to make sure that the Hadoop directory is available to
+all worker nodes which are intended to act as DataNodes and that all nodes
+**find the directory under the same path**. We recommend to use a shared network
+directory (e.g. an NFS share) for that. Alternatively, one can copy the
+directory to all nodes (with the disadvantage that all configuration and
+code updates need to be synced to all nodes).
+
+## Starting HDFS
+
+To start the HDFS log on to the master and type in the following
+commands
+
+```
+cd hadoop-*
+binn/start-dfs.sh
+```
+
+If your HDFS setup is correct, you should be able to open the HDFS
+status website at *http://MASTER:50070*. In a matter of a seconds,
+all DataNodes should appear as live nodes. For troubleshooting we would
+like to point you to the [Hadoop Quick
+Start](http://wiki.apache.org/hadoop/QuickStart)
+guide.
+
+# Stratosphere Setup
+
+Go to the [downloads page]({{site.baseurl}}/downloads/) and get the ready to run
+package. Make sure to pick the Stratosphere package **matching your Hadoop
+version**.
+
+After downloading the latest release, copy the archive to your master node and
+extract it:
+
+```
+tar xzf stratosphere-*.tgz
+cd stratosphere-*
+```
+
+## Configuring the Cluster
+
+After having extracted the system files, you need to configure Stratosphere for
+the cluster by editing *conf/stratosphere-conf.yaml*.
+
+Set the `jobmanager.rpc.address` key to point to your master node. Furthermode
+define the maximum amount of main memory the JVM is allowed to allocate on each
+node by setting the `jobmanager.heap.mb` and `taskmanager.heap.mb` keys.
+
+The value is given in MB. If some worker nodes have more main memory which you
+want to allocate to the Stratosphere system you can overwrite the default value
+by setting an environment variable `STRATOSPHERE_TM_HEAP` on the respective
+node.
+
+Finally you must provide a list of all nodes in your cluster which shall be used
+as worker nodes. Therefore, similar to the HDFS configuration, edit the file
+*conf/slaves* and enter the IP/host name of each worker node. Each worker node
+will later run a TaskManager.
+
+Each entry must be separated by a new line, as in the following example:
+
+```
+192.168.0.100
+192.168.0.101
+.
+.
+.
+192.168.0.150
+```
+
+The Stratosphere directory must be available on every worker under the same
+path. Similarly as for HDFS, you can use a shared NSF directory, or copy the
+entire Stratosphere directory to every worker node.
+
+## Configuring the Network Buffers
+
+Network buffers are a critical resource for the communication layers. They are
+used to buffer records before transmission over a network, and to buffer
+incoming data before dissecting it into records and handing them to the
+application. A sufficient number of network buffers are critical to achieve a
+good throughput.
+
+In general, configure the task manager to have so many buffers that each logical
+network connection on you expect to be open at the same time has a dedicated
+buffer. A logical network connection exists for each point-to-point exchange of
+data over the network, which typically happens at repartitioning- or
+broadcasting steps. In those, each parallel task inside the TaskManager has to
+be able to talk to all other parallel tasks. Hence, the required number of
+buffers on a task manager is *total-degree-of-parallelism* (number of targets)
+\* *intra-node-parallelism* (number of sources in one task manager) \* *n*.
+Here, *n* is a constant that defines how many repartitioning-/broadcasting steps
+you expect to be active at the same time.
+
+Since the *intra-node-parallelism* is typically the number of cores, and more
+than 4 repartitioning or broadcasting channels are rarely active in parallel, it
+frequently boils down to *\#cores\^2\^* \* *\#machines* \* 4. To support for
+example a cluster of 20 8-core machines, you should use roughly 5000 network
+buffers for optimal throughput.
+
+Each network buffer is by default 64 KiBytes large. In the above example, the
+system would allocate roughly 300 MiBytes for network buffers.
+
+The number and size of network buffers can be configured with the following
+parameters:
+
+- `taskmanager.network.numberOfBuffers`, and
+- `taskmanager.network.bufferSizeInBytes`.
+
+## Configuring Temporary I/O Directories
+
+Although Stratosphere aims to process as much data in main memory as possible,
+it is not uncommon that  more data needs to be processed than memory is
+available. Stratosphere's runtime is designed to  write temporary data to disk
+to handle these situations.
+
+The `taskmanager.tmp.dirs` parameter specifies a list of directories into which
+Stratosphere writes temporary files. The paths of the directories need to be
+separated by ':' (colon character).  Stratosphere will concurrently write (or
+read) one temporary file to (from) each configured directory.  This way,
+temporary I/O can be evenly distributed over multiple independent I/O devices
+such as hard disks to improve performance.  To leverage fast I/O devices (e.g.,
+SSD, RAID, NAS), it is possible to specify a directory multiple times.
+
+If the `taskmanager.tmp.dirs` parameter is not explicitly specified,
+Stratosphere writes temporary data to the temporary  directory of the operating
+system, such as */tmp* in Linux systems.
+
+Please see the [configuration page](config.html) for details and additional
+configuration options.
+
+## Starting Stratosphere
+
+The following script starts a JobManager on the local node and connects via
+SSH to all worker nodes listed in the *slaves* file to start the
+TaskManager on each node. Now your Stratosphere system is up and
+running. The JobManager running on the local node will now accept jobs
+at the configured RPC port.
+
+Assuming that you are on the master node and inside the Stratosphere directory:
+
+```
+bin/start-cluster.sh
+```

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/config.md
----------------------------------------------------------------------
diff --git a/docs/config.md b/docs/config.md
new file mode 100644
index 0000000..c11cc18
--- /dev/null
+++ b/docs/config.md
@@ -0,0 +1,171 @@
+---
+title:  "Configuration"
+---
+
+# Overview
+
+This page provides an overview of possible settings for Stratosphere. All
+configuration is done in `conf/stratosphere-conf.yaml`, which is expected to be
+a flat collection of [YAML key value pairs](http://www.yaml.org/spec/1.2/spec.html)
+with format `key: value`.
+
+The system and run scripts parse the config at startup and override the
+respective default values with the given values for every that has been set.
+This page contains a reference for all configuration keys used in the system.
+
+# Common Options
+
+- `env.java.home`: The path to the Java installation to use (DEFAULT: system's
+default Java installation).
+- `jobmanager.rpc.address`: The IP address of the JobManager (DEFAULT:
+localhost).
+- `jobmanager.rpc.port`: The port number of the JobManager (DEFAULT: 6123).
+- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
+(DEFAULT: 256).
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
+contrast to Hadoop, Stratosphere runs operators and functions inside the
+TaskManager (including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512).
+- `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
+directories separated by the systems directory delimiter (for example ':'
+(colon) on Linux/Unix). If multiple directories are specified then the temporary
+files will be distributed across the directories in a round robin fashion. The
+I/O manager component will spawn one reading and one writing thread per
+directory. A directory may be listed multiple times to have the I/O manager use
+multiple threads for it (for example if it is physically stored on a very fast
+disc or RAID) (DEFAULT: The system's tmp dir).
+- `parallelization.degree.default`: The default degree of parallelism to use for
+programs that have no degree of parallelism specified. A value of -1 indicates
+no limit, in which the degree of parallelism is set to the number of available
+instances at the time of compilation (DEFAULT: -1).
+- `parallelization.intra-node.default`: The number of parallel instances of an
+operation that are assigned to each TaskManager. A value of -1 indicates no
+limit (DEFAULT: -1).
+- `taskmanager.network.numberOfBuffers`: The number of buffers available to the
+network stack. This number determines how many streaming data exchange channels
+a TaskManager can have at the same time and how well buffered the channels are.
+If a job is rejected or you get a warning that the system has not enough buffers
+available, increase this value (DEFAULT: 2048).
+- `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves for sorting, hash tables, and caching of intermediate results.
+If unspecified (-1), the memory manager will take a fixed ratio of the heap
+memory available to the JVM after the allocation of the network buffers (0.8)
+(DEFAULT: -1).
+- `jobmanager.profiling.enable`: Flag to enable job manager's profiling
+component. This collects network/cpu utilization statistics, which are displayed
+as charts in the SWT visualization GUI (DEFAULT: false).
+
+# HDFS
+
+These parameters configure the default HDFS used by Stratosphere. If you don't
+specify a HDFS configuration, you will have to specify the full path to your
+HDFS files like `hdfs://address:port/path/to/files` and filed with be written
+with default HDFS parameters (block size, replication factor).
+
+- `fs.hdfs.hadoopconf`: The absolute path to the Hadoop configuration directory.
+The system will look for the "core-site.xml" and "hdfs-site.xml" files in that
+directory (DEFAULT: null).
+- `fs.hdfs.hdfsdefault`: The absolute path of Hadoop's own configuration file
+"hdfs-default.xml" (DEFAULT: null).
+- `fs.hdfs.hdfssite`: The absolute path of Hadoop's own configuration file
+"hdfs-site.xml" (DEFAULT: null).
+
+# JobManager &amp; TaskManager
+
+The following parameters configure Stratosphere's JobManager, TaskManager, and
+runtime channel management.
+
+- `jobmanager.rpc.address`: The hostname or IP address of the JobManager
+(DEFAULT: localhost).
+- `jobmanager.rpc.port`: The port of the JobManager (DEFAULT: 6123).
+- `jobmanager.rpc.numhandler`: The number of RPC threads for the JobManager.
+Increase those for large setups in which many TaskManagers communicate with the
+JobManager simultaneousl (DEFAULT: 8).
+- `jobmanager.profiling.enable`: Flag to enable the profiling component. This
+collects network/cpu utilization statistics, which are displayed as charts in
+the SWT visualization GUI. The profiling may add a small overhead on the
+execution (DEFAULT: false).
+- `jobmanager.web.port`: Port of the JobManager's web interface (DEFAULT: 8081).
+- `jobmanager.heap.mb`: JVM heap size (in megabytes) for the JobManager
+(DEFAULT: 256).
+- `taskmanager.heap.mb`: JVM heap size (in megabytes) for the TaskManager. In
+contrast to Hadoop, Stratosphere runs operators and functions inside the
+TaskManager (including sorting/hashing/caching), so this value should be as
+large as possible (DEFAULT: 512).
+- `taskmanager.rpc.port`: The task manager's IPC port (DEFAULT: 6122).
+- `taskmanager.data.port`: The task manager's port used for data exchange
+operations (DEFAULT: 6121).
+- `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
+directories separated by the systems directory delimiter (for example ':'
+(colon) on Linux/Unix). If multiple directories are specified then the temporary
+files will be distributed across the directories in a round robin fashion. The
+I/O manager component will spawn one reading and one writing thread per
+directory. A directory may be listed multiple times to have the I/O manager use
+multiple threads for it (for example if it is physically stored on a very fast
+disc or RAID) (DEFAULT: The system's tmp dir).
+- `taskmanager.network.numberOfBuffers`: The number of buffers available to the
+network stack. This number determines how many streaming data exchange channels
+a TaskManager can have at the same time and how well buffered the channels are.
+If a job is rejected or you get a warning that the system has not enough buffers
+available, increase this value (DEFAULT: 2048).
+- `taskmanager.network.bufferSizeInBytes`: The size of the network buffers, in
+bytes (DEFAULT: 32768 (= 32 KiBytes)).
+- `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
+manager reserves for sorting, hash tables, and caching of intermediate results.
+If unspecified (-1), the memory manager will take a relative amount of the heap
+memory available to the JVM after the allocation of the network buffers (0.8)
+(DEFAULT: -1).
+- `taskmanager.memory.fraction`: The fraction of memory (after allocation of the
+network buffers) that the task manager reserves for sorting, hash tables, and
+caching of intermediate results. This value is only used if
+'taskmanager.memory.size' is unspecified (-1) (DEFAULT: 0.8).
+- `jobclient.polling.interval`: The interval (in seconds) in which the client
+polls the JobManager for the status of its job (DEFAULT: 2).
+- `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and
+fan-out for spilling hash tables. Limits the numer of file handles per operator,
+but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128).
+- `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling
+when this fraction of its memory budget is full (DEFAULT: 0.8).
+- `taskmanager.runtime.fs_timeout`: The maximal time (in milliseconds) that the
+system waits for a response from the filesystem. Note that for HDFS, this time
+may occasionally be rather long. A value of 0 indicates infinite waiting time
+(DEFAULT: 0).
+
+# JobManager Web Frontend
+
+- `jobmanager.web.port`: Port of the JobManager's web interface that displays
+status of running jobs and execution time breakdowns of finished jobs
+(DEFAULT: 8081).
+- `jobmanager.web.history`: The number of latest jobs that the JobManager's web
+front-end in its history (DEFAULT: 5).
+
+# Webclient
+
+These parameters configure the web interface that can be used to submit jobs and
+review the compiler's execution plans.
+
+- `webclient.port`: The port of the webclient server (DEFAULT: 8080).
+- `webclient.tempdir`: The temp directory for the web server. Used for example
+for caching file fragments during file-uploads (DEFAULT: The system's temp
+directory).
+- `webclient.uploaddir`: The directory into which the web server will store
+uploaded programs (DEFAULT: ${webclient.tempdir}/webclient-jobs/).
+- `webclient.plandump`: The directory into which the web server will dump
+temporary JSON files describing the execution plans
+(DEFAULT: ${webclient.tempdir}/webclient-plans/).
+
+# Compiler/Optimizer
+
+- `compiler.delimited-informat.max-line-samples`: The maximum number of line
+samples taken by the compiler for delimited inputs. The samples are used to
+estimate the number of records. This value can be overridden for a specific
+input with the input format's parameters (DEFAULT: 10).
+- `compiler.delimited-informat.min-line-samples`: The minimum number of line
+samples taken by the compiler for delimited inputs. The samples are used to
+estimate the number of records. This value can be overridden for a specific
+input with the input format's parameters (DEFAULT: 2).
+- `compiler.delimited-informat.max-sample-len`: The maximal length of a line
+sample that the compiler takes for delimited inputs. If the length of a single
+sample exceeds this value (possible because of misconfiguration of the parser),
+the sampling aborts. This value can be overridden for a specific input with the
+input format's parameters (DEFAULT: 2097152 (= 2 MiBytes)).

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/css/syntax.css
----------------------------------------------------------------------
diff --git a/docs/css/syntax.css b/docs/css/syntax.css
new file mode 100644
index 0000000..2774b76
--- /dev/null
+++ b/docs/css/syntax.css
@@ -0,0 +1,60 @@
+.highlight  { background: #ffffff; }
+.highlight .c { color: #999988; font-style: italic } /* Comment */
+.highlight .err { color: #a61717; background-color: #e3d2d2 } /* Error */
+.highlight .k { font-weight: bold } /* Keyword */
+.highlight .o { font-weight: bold } /* Operator */
+.highlight .cm { color: #999988; font-style: italic } /* Comment.Multiline */
+.highlight .cp { color: #999999; font-weight: bold } /* Comment.Preproc */
+.highlight .c1 { color: #999988; font-style: italic } /* Comment.Single */
+.highlight .cs { color: #999999; font-weight: bold; font-style: italic } /* Comment.Special */
+.highlight .gd { color: #000000; background-color: #ffdddd } /* Generic.Deleted */
+.highlight .gd .x { color: #000000; background-color: #ffaaaa } /* Generic.Deleted.Specific */
+.highlight .ge { font-style: italic } /* Generic.Emph */
+.highlight .gr { color: #aa0000 } /* Generic.Error */
+.highlight .gh { color: #999999 } /* Generic.Heading */
+.highlight .gi { color: #000000; background-color: #ddffdd } /* Generic.Inserted */
+.highlight .gi .x { color: #000000; background-color: #aaffaa } /* Generic.Inserted.Specific */
+.highlight .go { color: #888888 } /* Generic.Output */
+.highlight .gp { color: #555555 } /* Generic.Prompt */
+.highlight .gs { font-weight: bold } /* Generic.Strong */
+.highlight .gu { color: #aaaaaa } /* Generic.Subheading */
+.highlight .gt { color: #aa0000 } /* Generic.Traceback */
+.highlight .kc { font-weight: bold } /* Keyword.Constant */
+.highlight .kd { font-weight: bold } /* Keyword.Declaration */
+.highlight .kp { font-weight: bold } /* Keyword.Pseudo */
+.highlight .kr { font-weight: bold } /* Keyword.Reserved */
+.highlight .kt { color: #445588; font-weight: bold } /* Keyword.Type */
+.highlight .m { color: #009999 } /* Literal.Number */
+.highlight .s { color: #d14 } /* Literal.String */
+.highlight .na { color: #008080 } /* Name.Attribute */
+.highlight .nb { color: #0086B3 } /* Name.Builtin */
+.highlight .nc { color: #445588; font-weight: bold } /* Name.Class */
+.highlight .no { color: #008080 } /* Name.Constant */
+.highlight .ni { color: #800080 } /* Name.Entity */
+.highlight .ne { color: #990000; font-weight: bold } /* Name.Exception */
+.highlight .nf { color: #990000; font-weight: bold } /* Name.Function */
+.highlight .nn { color: #555555 } /* Name.Namespace */
+.highlight .nt { color: #000080 } /* Name.Tag */
+.highlight .nv { color: #008080 } /* Name.Variable */
+.highlight .ow { font-weight: bold } /* Operator.Word */
+.highlight .w { color: #bbbbbb } /* Text.Whitespace */
+.highlight .mf { color: #009999 } /* Literal.Number.Float */
+.highlight .mh { color: #009999 } /* Literal.Number.Hex */
+.highlight .mi { color: #009999 } /* Literal.Number.Integer */
+.highlight .mo { color: #009999 } /* Literal.Number.Oct */
+.highlight .sb { color: #d14 } /* Literal.String.Backtick */
+.highlight .sc { color: #d14 } /* Literal.String.Char */
+.highlight .sd { color: #d14 } /* Literal.String.Doc */
+.highlight .s2 { color: #d14 } /* Literal.String.Double */
+.highlight .se { color: #d14 } /* Literal.String.Escape */
+.highlight .sh { color: #d14 } /* Literal.String.Heredoc */
+.highlight .si { color: #d14 } /* Literal.String.Interpol */
+.highlight .sx { color: #d14 } /* Literal.String.Other */
+.highlight .sr { color: #009926 } /* Literal.String.Regex */
+.highlight .s1 { color: #d14 } /* Literal.String.Single */
+.highlight .ss { color: #990073 } /* Literal.String.Symbol */
+.highlight .bp { color: #999999 } /* Name.Builtin.Pseudo */
+.highlight .vc { color: #008080 } /* Name.Variable.Class */
+.highlight .vg { color: #008080 } /* Name.Variable.Global */
+.highlight .vi { color: #008080 } /* Name.Variable.Instance */
+.highlight .il { color: #009999 } /* Literal.Number.Integer.Long */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
new file mode 100644
index 0000000..3ceb527
--- /dev/null
+++ b/docs/faq.md
@@ -0,0 +1,285 @@
+---
+title: "Frequently Asked Questions (FAQ)"
+---
+
+# General
+
+## Is Stratosphere a Hadoop Project?
+
+Stratosphere is a data processing system and an alternative to Hadoop's
+MapReduce component. It comes with its own runtime, rather than building on top
+of MapReduce. As such, it can work completely independently of the Hadoop
+ecosystem. However, Stratosphere can also access Hadoop's distributed file
+system (HDFS) to read and write data, and Hadoop's next-generation resource
+manager (YARN) to provision cluster resources. Since most Stratosphere users are
+using Hadoop HDFS to store their data, we ship already the required libraries to
+access HDFS.
+
+## Do I have to install Apache Hadoop to use Stratosphere?
+
+No. Stratosphere can run without a Hadoop installation. However, a very common
+setup is to use Stratosphere to analyze data stored in the Hadoop Distributed
+File System (HDFS). To make these setups work out of the box, we bundle the
+Hadoop client libraries with Stratosphere by default.
+
+Additionally, we provide a special YARN Enabled download of Stratosphere for
+users with an existing Hadoop YARN cluster. [Apache Hadoop
+YARN](http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-
+site/YARN.html) is Hadoop's cluster resource manager that allows to use
+different execution engines next to each other on a cluster.
+
+# Usage
+
+## How do I assess the progress of a Stratosphere program?
+
+There are a multiple of ways to track the progress of a Stratosphere program:
+
+- The JobManager (the master of the distributed system) starts a web interface
+to observe program execution. In runs on port 8081 by default (configured in
+`conf/stratosphere-config.yml`).
+- When you start a program from the command line, it will print the status
+changes of all operators as the program progresses through the operations.
+- All status changes are also logged to the JobManager's log file.
+
+## How can I figure out why a program failed?
+
+- Thw JobManager web frontend (by default on port 8081) displays the exceptions
+of failed tasks.
+- If you run the program from the command-line, task exceptions are printed to
+the standard error stream and shown on the console.
+- Both the command line and the web interface allow you to figure out which
+parallel task first failed and caused the other tasks to cancel the execution.
+- Failing tasks and the corresponding exceptions are reported in the log files
+of the master and the worker where the exception occurred
+(`log/stratosphere-<user>-jobmanager-<host>.log` and
+`log/stratosphere-<user>-taskmanager-<host>.log`).
+
+## How do I debug Stratosphere programs?
+
+- When you start a program locally with the [LocalExecutor](local_execution.html),
+you can place breakpoints in your functions and debug them like normal
+Java/Scala programs.
+- The [Accumulators](java_api_guide.html#accumulators) are very helpful in
+tracking the behavior of the parallel execution. They allow you to gather
+information inside the program's operations and show them after the program
+execution.
+
+# Errors
+
+## I get an error message saying that not enough buffers are available. How do I fix this?
+
+If you run Stratosphere in a massively parallel setting (100+ parallel threads),
+you need to adapt the number of network buffers via the config parameter
+`taskmanager.network.numberOfBuffers`.
+As a rule-of-thumb, the number of buffers should be at least
+`4 * numberOfNodes * numberOfTasksPerNode^2`. See
+[Configuration Reference](config.html) for details.
+
+## My job fails early with a java.io.EOFException. What could be the cause?
+
+Note: In version <em>0.4</em>, the delta iterations limit the solution set to
+records with fixed-length data types. We will  in the next version.
+
+The most common case for these exception is when Stratosphere is set up with the
+wrong HDFS version. Because different HDFS versions are often not compatible
+with each other, the connection between the filesystem master and the client
+breaks.
+
+```bash
+Call to <host:port> failed on local exception: java.io.EOFException
+    at org.apache.hadoop.ipc.Client.wrapException(Client.java:775)
+    at org.apache.hadoop.ipc.Client.call(Client.java:743)
+    at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:220)
+    at $Proxy0.getProtocolVersion(Unknown Source)
+    at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:359)
+    at org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:106)
+    at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:207)
+    at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:170)
+    at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:82)
+    at eu.stratosphere.runtime.fs.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:276
+```
+
+Please refer to the [download page](http://stratosphere.eu/downloads/#maven) and
+the [build instructions](https://github.com/stratosphere/stratosphere/blob/master/README.md)
+for details on how to set up Stratosphere for different Hadoop and HDFS versions.
+
+## My program does not compute the correct result. Why are my custom key types
+are not grouped/joined correctly?
+
+Keys must correctly implement the methods `java.lang.Object#hashCode()`,
+`java.lang.Object#equals(Object o)`, and `java.util.Comparable#compareTo(...)`.
+These methods are always backed with default implementations which are usually
+inadequate. Therefore, all keys must override `hashCode()` and `equals(Object o)`.
+
+## I get a java.lang.InstantiationException for my data type, what is wrong?
+
+All data type classes must be public and have a public nullary constructor
+(constructor with no arguments). Further more, the classes must not be abstract
+or interfaces. If the classes are internal classes, they must be public and
+static.
+
+## I can't stop Stratosphere with the provided stop-scripts. What can I do?
+
+Stopping the processes sometimes takes a few seconds, because the shutdown may
+do some cleanup work.
+
+In some error cases it happens that the JobManager or TaskManager cannot be
+stopped with the provided stop-scripts (`bin/stop-local.sh` or `bin/stop-
+cluster.sh`). You can kill their processes on Linux/Mac as follows:
+
+- Determine the process id (pid) of the JobManager / TaskManager process. You
+can use the `jps` command on Linux(if you have OpenJDK installed) or command
+`ps -ef | grep java` to find all Java processes. 
+- Kill the process with `kill -9 <pid>`, where `pid` is the process id of the
+affected JobManager or TaskManager process.
+    
+On Windows, the TaskManager shows a table of all processes and allows you to
+destroy a process by right its entry.
+
+## I got an OutOfMemoryException. What can I do?
+
+These exceptions occur usually when the functions in the program consume a lot
+of memory by collection large numbers of objects, for example in lists or maps.
+The OutOfMemoryExceptions in Java are kind of tricky. The exception is not
+necessarily thrown by the component that allocated most of the memory but by the
+component that tried to requested the latest bit of memory that could not be
+provided.
+
+There are two ways to go about this:
+
+1. See whether you can use less memory inside the functions. For example, use
+arrays of primitive types instead of object types.
+
+2. Reduce the memory that Stratosphere reserves for its own processing. The
+TaskManager reserves a certain portion of the available memory for sorting,
+hashing, caching, network buffering, etc. That part of the memory is unavailable
+to the user-defined functions. By reserving it, the system can guarantee to not
+run out of memory on large inputs, but to plan with the available memory and
+destage operations to disk, if necessary. By default, the system reserves around
+70% of the memory. If you frequently run applications that need more memory in
+the user-defined functions, you can reduce that value using the configuration
+entries `taskmanager.memory.fraction` or `taskmanager.memory.size`. See the
+[Configuration Reference](http://stratosphere.eu/docs/0.4/setup/config.html
+"Configuration Reference") for details. This will leave more memory to JVM heap,
+but may cause data processing tasks to go to disk more often.
+
+## Why do the TaskManager log files become so huge?
+
+Check the logging behavior of your jobs. Emitting logging per or tuple may be
+helpful to debug jobs in small setups with tiny data sets, it becomes very
+inefficient and disk space consuming if used for large input data.
+
+# YARN Deployment
+
+## The YARN session runs only for a few seconds
+
+The `./bin/yarn-session.sh` script is intended to run while the YARN-session is
+open. In some error cases however, the script immediately stops running. The
+output looks like this:
+
+```
+07:34:27,004 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Submitted application application_1395604279745_273123 to ResourceManager at jobtracker-host
+Stratosphere JobManager is now running on worker1:6123
+JobManager Web Interface: http://jobtracker-host:54311/proxy/application_1295604279745_273123/
+07:34:51,528 INFO  eu.stratosphere.yarn.Client                                   - Application application_1295604279745_273123 finished with state FINISHED at 1398152089553
+07:34:51,529 INFO  eu.stratosphere.yarn.Client                                   - Killing the Stratosphere-YARN application.
+07:34:51,529 INFO  org.apache.hadoop.yarn.client.api.impl.YarnClientImpl         - Killing application application_1295604279745_273123
+07:34:51,534 INFO  eu.stratosphere.yarn.Client                                   - Deleting files in hdfs://user/marcus/.stratosphere/application_1295604279745_273123
+07:34:51,559 INFO  eu.stratosphere.yarn.Client                                   - YARN Client is shutting down
+```
+
+The problem here is that the Application Master (AM) is stopping and the YARN client assumes that the application has finished.
+
+There are three possible reasons for that behavior:
+
+- The ApplicationMaster exited with an exception. To debug that error, have a
+look in the logfiles of the container. The `yarn-site.xml` file contains the
+configured path. The key for the path is `yarn.nodemanager.log-dirs`, the
+default value is `${yarn.log.dir}/userlogs`.
+
+- YARN has killed the container that runs the ApplicationMaster. This case
+happens when the AM used too much memory or other resources beyond YARN's
+limits. In this case, you'll find error messages in the nodemanager logs on
+the host.
+
+- The operating system has shut down the JVM of the AM. This can happen if the
+YARN configuration is wrong and more memory than physically available is
+configured. Execute `dmesg` on the machine where the AM was running to see if
+this happened. You see messages from Linux' [OOM killer](http://linux-mm.org/OOM_Killer).
+
+## The YARN session crashes with a HDFS permission exception during startup
+
+While starting the YARN session, you are receiving an exception like this:
+
+```
+Exception in thread "main" org.apache.hadoop.security.AccessControlException: Permission denied: user=robert, access=WRITE, inode="/user/robert":hdfs:supergroup:drwxr-xr-x
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:234)
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:214)
+  at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:158)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:5193)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:5175)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkAncestorAccess(FSNamesystem.java:5149)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2090)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2043)
+  at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:1996)
+  at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:491)
+  at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:301)
+  at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:59570)
+  at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
+  at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928)
+  at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2053)
+  at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
+  at java.security.AccessController.doPrivileged(Native Method)
+  at javax.security.auth.Subject.doAs(Subject.java:396)
+  at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
+  at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2047)
+
+  at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
+  at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
+  at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
+  at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
+  at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
+  at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
+  at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1393)
+  at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1382)
+  at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1307)
+  at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:384)
+  at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:380)
+  at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
+  at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:380)
+  at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:324)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:905)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:886)
+  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:783)
+  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:365)
+  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:338)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:2021)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1989)
+  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1954)
+  at eu.stratosphere.yarn.Utils.setupLocalResource(Utils.java:176)
+  at eu.stratosphere.yarn.Client.run(Client.java:362)
+  at eu.stratosphere.yarn.Client.main(Client.java:568)
+```
+
+The reason for this error is, that the home directory of the user **in HDFS**
+has the wrong permissions. The user (in this case `robert`) can not create
+directories in his own home directory.
+
+Stratosphere creates a `.stratosphere/` directory in the users home directory
+where it stores the Stratosphere jar and configuration file.
+
+# Features
+
+## What kind of fault-tolerance does Stratosphere provide?
+
+Stratospere can restart failed jobs. Mid-query fault tolerance will go into the
+open source project in the next versions.
+
+## Are Hadoop-like utilities, such as Counters and the DistributedCache supported?
+
+[Stratosphere's Accumulators](java_api_guide.html) work very similar like
+[Hadoop's counters, but are more powerful.
+
+Stratosphere has a [Distributed Cache](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-core/src/main/java/eu/stratosphere/api/common/cache/DistributedCache.java) that is deeply integrated with the APIs. Please refer to the [JavaDocs](https://github.com/stratosphere/stratosphere/blob/{{ site.docs_05_stable_gh_tag }}/stratosphere-java/src/main/java/eu/stratosphere/api/java/ExecutionEnvironment.java#L561) for details on how to use it.
+
+In order to make data sets available on all tasks, we encourage you to use [Broadcast Variables]({{site.baseurl}}/docs/0.5/programming_guides/java.html#broadcast_variables) instead. They are more efficient and easier to use than the distributed cache.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/hadoop_compatability.md
----------------------------------------------------------------------
diff --git a/docs/hadoop_compatability.md b/docs/hadoop_compatability.md
new file mode 100644
index 0000000..06c0dfa
--- /dev/null
+++ b/docs/hadoop_compatability.md
@@ -0,0 +1,5 @@
+---
+title: "Hadoop Compatability"
+---
+
+To be written.
\ No newline at end of file


[5/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/dataflow.svg
----------------------------------------------------------------------
diff --git a/docs/img/dataflow.svg b/docs/img/dataflow.svg
new file mode 100644
index 0000000..07d6e31
--- /dev/null
+++ b/docs/img/dataflow.svg
@@ -0,0 +1,979 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="248"
+   id="svg10359"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="dataflow.svg">
+  <defs
+     id="defs10361">
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path11612"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lend"
+       style="overflow:visible">
+      <path
+         id="path11609"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.8,0,0,-0.8,-10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-7"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-4"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-5"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-47"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-8"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-32"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-0"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-4"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-1"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-8"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-7"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-3"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-37"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-82"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-05"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-5"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-03"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-00"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-6"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-84"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-1"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-6"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-42"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-07"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-74"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-32"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-89"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-80"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-77"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-69"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-79"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-03"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-41"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-02"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart-45"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path11612-01"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="250.86321"
+     inkscape:cy="254.0586"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata10364">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-101.14286,-83.672699)">
+    <g
+       id="g12838"
+       transform="translate(94.357143,-2.3120884)">
+      <g
+         transform="translate(-10.618665,-105.0985)"
+         id="g11342">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(16.33296,-1.5382389)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="147.0238"
+           y="223.23857"
+           id="text10541"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543"
+             x="147.0238"
+             y="223.23857"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="147.0238"
+             y="243.23857"
+             id="tspan10545"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(-126.14284,-10.618054)"
+         id="g11348">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-2"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(240.00001,78.928564)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="370.69086"
+           y="303.70538"
+           id="text10541-6"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543-6"
+             x="370.69086"
+             y="303.70538"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="370.69086"
+             y="323.70538"
+             id="tspan10545-9"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(-69.857101,-112.71429)"
+         id="g11503">
+        <g
+           id="g11348-9"
+           transform="translate(160,98.571427)">
+          <path
+             transform="translate(240.00001,78.928564)"
+             d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+             sodipodi:ry="33.214287"
+             sodipodi:rx="34.285713"
+             sodipodi:cy="229.1479"
+             sodipodi:cx="130.71428"
+             id="path10539-2-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+             sodipodi:type="arc" />
+          <text
+             sodipodi:linespacing="125%"
+             id="text10541-6-8"
+             y="303.70538"
+             x="370.69086"
+             style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+             xml:space="preserve"><tspan
+               style="text-align:center;text-anchor:middle"
+               y="303.70538"
+               x="370.69086"
+               id="tspan10543-6-9"
+               sodipodi:role="line">Data</tspan><tspan
+               style="text-align:center;text-anchor:middle"
+               id="tspan10545-9-8"
+               y="323.70538"
+               x="370.69086"
+               sodipodi:role="line">Source</tspan></text>
+        </g>
+      </g>
+      <g
+         transform="translate(-84.847654,-64.285714)"
+         id="g11488">
+        <path
+           transform="translate(198.70482,-42.351028)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-4"
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="313.57144"
+           y="190.93359"
+           id="text11423"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11425"
+             x="313.57144"
+             y="190.93359">Map</tspan></text>
+      </g>
+      <g
+         transform="translate(22.684718,-61.12656)"
+         id="g11483">
+        <path
+           transform="translate(199.31529,86.152908)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-08"
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="302.14285"
+           y="320.93362"
+           id="text11427"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11429"
+             x="302.14285"
+             y="320.93362">Reduce</tspan></text>
+      </g>
+      <g
+         transform="translate(5.4420293,-37.982104)"
+         id="g11493">
+        <path
+           transform="translate(324.70088,19.724304)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-7"
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="433.57141"
+           y="254.50502"
+           id="text11431"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11433"
+             x="433.57141"
+             y="254.50502">Match</tspan></text>
+      </g>
+      <g
+         transform="translate(-4.272862,3.5697637)"
+         id="g11498">
+        <path
+           transform="translate(442.55859,19.694183)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-0"
+           style="fill:#87d69b;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="540.71429"
+           y="253.07646"
+           id="text11450"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11452"
+             x="540.71429"
+             y="253.07646">CoGroup</tspan></text>
+      </g>
+      <g
+         transform="translate(24.571445,-118.61493)"
+         id="g11478">
+        <path
+           transform="translate(89.285721,100.35713)"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           sodipodi:ry="33.214287"
+           sodipodi:rx="34.285713"
+           sodipodi:cy="229.1479"
+           sodipodi:cx="130.71428"
+           id="path10539-4-2"
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           sodipodi:type="arc" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="204.15234"
+           y="333.64175"
+           id="text11423-8"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11425-8"
+             x="204.15234"
+             y="333.64175">Map</tspan></text>
+      </g>
+      <g
+         transform="translate(146.42857,-154.23604)"
+         id="g11503-1">
+        <g
+           id="g11348-9-6"
+           transform="translate(160,98.571427)">
+          <path
+             transform="translate(240.00001,78.928564)"
+             d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+             sodipodi:ry="33.214287"
+             sodipodi:rx="34.285713"
+             sodipodi:cy="229.1479"
+             sodipodi:cx="130.71428"
+             id="path10539-2-2-4"
+             style="fill:#e1f5c4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+             sodipodi:type="arc" />
+          <text
+             sodipodi:linespacing="125%"
+             id="text10541-6-8-2"
+             y="303.70538"
+             x="370.69086"
+             style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+             xml:space="preserve"><tspan
+               style="text-align:center;text-anchor:middle"
+               y="303.70538"
+               x="370.69086"
+               id="tspan10543-6-9-6"
+               sodipodi:role="line">Data</tspan><tspan
+               style="text-align:center;text-anchor:middle"
+               id="tspan10545-9-8-8"
+               y="323.70538"
+               x="370.69086"
+               sodipodi:role="line">Sink</tspan></text>
+        </g>
+      </g>
+      <g
+         transform="translate(-10.618665,-16.719561)"
+         id="g11342-0">
+        <path
+           sodipodi:type="arc"
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-20"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(16.33296,-1.5382389)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="147.0238"
+           y="223.23857"
+           id="text10541-9"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan10543-0"
+             x="147.0238"
+             y="223.23857"
+             style="text-align:center;text-anchor:middle">Data</tspan><tspan
+             sodipodi:role="line"
+             x="147.0238"
+             y="243.23857"
+             id="tspan10545-2"
+             style="text-align:center;text-anchor:middle">Source</tspan></text>
+      </g>
+      <g
+         transform="translate(26.003898,-327.17553)"
+         id="g11595">
+        <path
+           sodipodi:type="arc"
+           style="fill:#93cbf8;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="path10539-7-3"
+           sodipodi:cx="130.71428"
+           sodipodi:cy="229.1479"
+           sodipodi:rx="34.285713"
+           sodipodi:ry="33.214287"
+           d="m 164.99999,229.1479 c 0,18.34375 -15.35023,33.21429 -34.28571,33.21429 -18.93548,0 -34.285714,-14.87054 -34.285714,-33.21429 0,-18.34374 15.350234,-33.21428 34.285714,-33.21428 18.93548,0 34.28571,14.87054 34.28571,33.21428 z"
+           transform="translate(195.99611,264.72826)" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="305.71429"
+           y="499.50507"
+           id="text11591"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             id="tspan11593"
+             x="305.71429"
+             y="499.50507">Union</tspan></text>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600"
+         d="m 207.29525,122.36218 -36.6524,0"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600-3"
+         d="m 207.61192,210.8901 -36.6524,0"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5"
+         d="M 316.50477,153.07647 276.10269,135.86026"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30"
+         d="m 318.50402,182.481 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         inkscape:connector-curvature="0"
+         id="path11600-3-1"
+         d="m 640.16549,252.41185 -36.65241,0"
+         style="fill:none;stroke:#333333;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5-4"
+         d="M 425.80414,196.37869 384.84328,179.32635"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-5-4-6"
+         d="M 535.14863,238.16024 493.24884,221.04335"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7"
+         d="m 425.9661,225.06044 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7-0"
+         d="m 317.69053,267.33329 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+      <path
+         sodipodi:nodetypes="cc"
+         inkscape:connector-curvature="0"
+         id="path11600-30-7-7"
+         d="m 537.22878,272.70837 -41.52749,16.69655"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:url(#Arrow1Mstart)" />
+    </g>
+    <g
+       id="g12953"
+       style="fill:#cccccc;stroke-width:5.2870717;stroke-miterlimit:4;stroke-dasharray:none"
+       transform="matrix(0.41814798,0,0,0.40644285,558.62945,282.71148)">
+      <path
+         id="path12920-9"
+         d="m -278.8125,-54.03125 0,53.625 c -0.0646,0.44696162 -0.0937,0.92003723 -0.0937,1.375 0,10.49341 17.46089,19 39,19 21.49586,0 38.93008,-8.473146 39,-18.9375 l 0.0937,0 0,-55.0625 -78,0 z"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:4.85137939;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+         inkscape:connector-curvature="0"
+         transform="translate(101.14286,83.672699)" />
+      <path
+         transform="matrix(1.0434603,0,0,1.3435029,44.746165,-114.27442)"
+         d="m -138.3909,105.56849 c 0,7.81048 -16.73364,14.14213 -37.37564,14.14213 -20.642,0 -37.37565,-6.33165 -37.37565,-14.14213 0,-7.810487 16.73365,-14.142137 37.37565,-14.142137 20.642,0 37.37564,6.33165 37.37564,14.142137 z"
+         sodipodi:ry="14.142136"
+         sodipodi:rx="37.375645"
+         sodipodi:cy="105.56849"
+         sodipodi:cx="-175.76654"
+         id="path12920"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:4.09739685;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+         sodipodi:type="arc" />
+    </g>
+    <g
+       id="g13048"
+       transform="matrix(0.53768785,0,0,0.6237725,211.71852,24.496616)">
+      <g
+         id="g12998">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13048-1"
+       transform="matrix(0.53768785,0,0,0.6237725,211.80554,112.87556)">
+      <g
+         id="g12998-3">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-3"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-04"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8-8"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82-0"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4-0"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0-3"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1-0"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1-7"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6-0"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13048-3"
+       transform="matrix(0.53768785,0,0,0.6237725,319.87156,199.44387)">
+      <g
+         id="g12998-87">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-48"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-2"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-8-86"
+         transform="translate(-5.7638991,-4.842059)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-0-2"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-5-3"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-82-6"
+         transform="translate(-11.5278,-9.6840945)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-4-7"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-0-7"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+      <g
+         id="g12998-1-6"
+         transform="translate(-17.291695,-14.526155)">
+        <path
+           style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:3.45343852;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0"
+           d="m -83.252987,150.6069 46.538697,0 0,46.65177 0,9.41403 -10.803621,0 -35.735076,0 z"
+           id="rect12981-1-8"
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="ccccccc" />
+        <path
+           style="fill:none;stroke:#333333;stroke-width:3.45343876;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           d="m -150.90663,122.80518 6.55098,-10.69463 6.44621,4.69223"
+           id="path12984-6-1"
+           inkscape:connector-curvature="0"
+           transform="translate(101.14286,83.672699)"
+           sodipodi:nodetypes="ccc" />
+      </g>
+    </g>
+    <g
+       id="g13190"
+       transform="translate(447.14285,-100.13051)">
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="rect12981-1-8-1"
+         d="m 363.80979,348.47226 25.02329,0 0,29.10009 0,5.87221 -5.80897,0 -19.21432,0 z"
+         style="fill:#cccccc;fill-opacity:1;stroke:#333333;stroke-width:1.99999976;stroke-miterlimit:4;stroke-opacity:1;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="ccc"
+         inkscape:connector-curvature="0"
+         id="path12984-6-1-4"
+         d="m 381.81653,383.32304 3.52238,-6.67102 3.46605,2.92689"
+         style="fill:none;stroke:#333333;stroke-width:1.99999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/datatypes.svg
----------------------------------------------------------------------
diff --git a/docs/img/datatypes.svg b/docs/img/datatypes.svg
new file mode 100644
index 0000000..a949677
--- /dev/null
+++ b/docs/img/datatypes.svg
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="85.809998"
+   id="svg5014"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="datatypes.svg">
+  <defs
+     id="defs5016" />
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.7"
+     inkscape:cx="-61.192684"
+     inkscape:cy="64.192611"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     inkscape:snap-global="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     showguides="true"
+     inkscape:guide-bbox="true"
+     inkscape:window-width="1916"
+     inkscape:window-height="747"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata5019">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-110.75,-300.24867)">
+    <g
+       id="g5646"
+       transform="translate(223.38524,-2.9974298)">
+      <rect
+         y="307.24353"
+         x="113.73705"
+         height="28.124744"
+         width="28.797562"
+         id="rect2985-5-8"
+         style="fill:#666666;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <rect
+         y="345.68823"
+         x="113.73705"
+         height="28.124744"
+         width="28.797562"
+         id="rect2985-5-4"
+         style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text4753"
+         y="331.45355"
+         x="147.99387"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:14.64699078px"
+           id="tspan4757"
+           y="331.45355"
+           x="147.99387"
+           sodipodi:role="line"><tspan
+   id="tspan5642"
+   style="font-size:16px">Value</tspan>:</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         inkscape:transform-center-y="24.124547"
+         inkscape:transform-center-x="73.131734"
+         sodipodi:linespacing="125%"
+         id="text4827"
+         y="369.04004"
+         x="147.99387"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:14.64699078px"
+           y="369.04004"
+           x="147.99387"
+           id="tspan4829"
+           sodipodi:role="line"><tspan
+   id="tspan5644"
+   style="font-size:16px">Key</tspan>:</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text5588"
+         y="331.45355"
+         x="195.89445"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Courier New;-inkscape-font-specification:Courier New"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="331.45355"
+           x="195.89445"
+           id="tspan5590"
+           sodipodi:role="line">Serialize(), Deserialize()</tspan></text>
+      <text
+         transform="scale(1.0155339,0.98470371)"
+         sodipodi:linespacing="125%"
+         id="text5588-5"
+         y="369.04004"
+         x="195.89445"
+         style="font-size:17.57638931px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Courier New;-inkscape-font-specification:Courier New"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="369.04004"
+           x="195.89445"
+           id="tspan5590-2"
+           sodipodi:role="line">Serialize(), Deserialize(), </tspan><tspan
+           id="tspan5613"
+           style="font-size:16px"
+           y="389.04004"
+           x="195.89445"
+           sodipodi:role="line">Hash(),      Compare(Key)</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_delta_iterate_operator.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_delta_iterate_operator.png b/docs/img/iterations_delta_iterate_operator.png
new file mode 100644
index 0000000..470485a
Binary files /dev/null and b/docs/img/iterations_delta_iterate_operator.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_delta_iterate_operator_example.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_delta_iterate_operator_example.png b/docs/img/iterations_delta_iterate_operator_example.png
new file mode 100644
index 0000000..15f2b54
Binary files /dev/null and b/docs/img/iterations_delta_iterate_operator_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_iterate_operator.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_iterate_operator.png b/docs/img/iterations_iterate_operator.png
new file mode 100644
index 0000000..aaf4158
Binary files /dev/null and b/docs/img/iterations_iterate_operator.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_iterate_operator_example.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_iterate_operator_example.png b/docs/img/iterations_iterate_operator_example.png
new file mode 100644
index 0000000..be4841c
Binary files /dev/null and b/docs/img/iterations_iterate_operator_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/iterations_supersteps.png
----------------------------------------------------------------------
diff --git a/docs/img/iterations_supersteps.png b/docs/img/iterations_supersteps.png
new file mode 100644
index 0000000..331dbc7
Binary files /dev/null and b/docs/img/iterations_supersteps.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/japi_example_overview.png
----------------------------------------------------------------------
diff --git a/docs/img/japi_example_overview.png b/docs/img/japi_example_overview.png
new file mode 100644
index 0000000..c035e3b
Binary files /dev/null and b/docs/img/japi_example_overview.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/join.svg
----------------------------------------------------------------------
diff --git a/docs/img/join.svg b/docs/img/join.svg
new file mode 100644
index 0000000..d5ef018
--- /dev/null
+++ b/docs/img/join.svg
@@ -0,0 +1,615 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="match.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-5"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-2"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="3.959798"
+     inkscape:cx="236.8846"
+     inkscape:cy="139.10312"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="374"
+     inkscape:window-x="1400"
+     inkscape:window-y="412"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g35027"
+       transform="translate(1951.0537,-629.64646)">
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-0-7"
+         d="m -938.8172,909.44427 0,23.67486"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <rect
+         y="932.67249"
+         x="-1223.4713"
+         height="136.06552"
+         width="57.281906"
+         id="rect2994-5-9-0-0"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-1-6"
+         d="m -1174.1847,1054.5519 32.1253,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.59638965;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40774172,0,0,0.41277371,-1301.4257,1623.5379)"
+         id="g3792-4-1-7-5">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65630722;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-3-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630722;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-1-7"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-1-2"
+         transform="matrix(0.40774188,0,0,0.41277387,-1301.4258,1658.2654)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-9-8"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-0-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-5-2"
+         transform="matrix(0.40454759,0,0,0.40954016,-1300.5907,1722.3573)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.6851759;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-6-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.6851759;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-7-5"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-7-4"
+         transform="matrix(0.40560715,0,0,0.41061279,-1300.8677,1689.4088)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67554951;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-4-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67554951;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-0-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="881.41754"
+         x="-1141.5946"
+         height="32.721252"
+         width="232.56281"
+         id="rect2994-5-9-4-2-2"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40774188,0,0,0.41277387,-1045.3506,1572.0715)"
+         id="g3792-4-1-1-6-7">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-47-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65630603;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-4-8"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-8-8"
+         transform="matrix(0.39966779,0,0,0.40460014,-1161.2482,1558.5157)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.73017049;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-5-7"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.73017049;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-8-4"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cccc"
+         id="path4066-2-0-78-4"
+         d="m -1174.4433,949.28372 112.1612,-0.58473 c 0,-7.97392 10.4003,-7.71183 10.4003,0.2622 l 28.6711,0.32253"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cccc"
+         id="path4066-2-0-7-4-2"
+         d="m -1174.5501,1018.6844 112.2161,-0.597 c 0,-8.0094 10.5429,-7.991 10.5429,0 l 28.5804,0.5971"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-9-8"
+         d="m -1056.6686,907.98234 -0.3925,129.59686"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.51504707;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-0-8-8"
+         d="m -938.7653,907.97323 0.1239,94.69557"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.51167059;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         transform="translate(426.5,-0.5)"
+         id="g34412">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-6-8"
+           width="103.5"
+           height="28.5"
+           x="-1562.7062"
+           y="1041.1628" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34070-6">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-1-17-4-6-9"
+             transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-8-4-1-8-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-5-3-2-1-8"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-2-0-2-3-7-5-9"
+             transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-2-4-5-3-9-5-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-4-8-4-7-60-6-3"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(428.43934,-1.3232233)"
+         id="g34422">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-5-0"
+           width="103.5"
+           height="28.5"
+           x="-1446.7938"
+           y="937.06146" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34078-7">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-0">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-1"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-4"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-1">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-7"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-4"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(428.60562,67.816701)"
+         id="g34422-3">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-5-0-0"
+           width="103.5"
+           height="28.5"
+           x="-1446.7938"
+           y="937.06146" />
+        <g
+           transform="translate(-391.67928,287.33529)"
+           id="g34078-7-9">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-0-9">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-1-1"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-4-4"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-1-8">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-7-9"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-4-9"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+    </g>
+    <path
+       inkscape:connector-curvature="0"
+       sodipodi:nodetypes="ccc"
+       id="path4066-2-0-9-4-1-2"
+       d="m 723.38563,271.07503 22.75265,-0.035 -0.12654,35.13393"
+       style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    <text
+       xml:space="preserve"
+       style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       x="655.62878"
+       y="275.41705"
+       id="text39591"
+       sodipodi:linespacing="125%"><tspan
+         sodipodi:role="line"
+         id="tspan39593"
+         x="655.62878"
+         y="275.41705">Key Field</tspan></text>
+  </g>
+</svg>


[4/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/map.svg
----------------------------------------------------------------------
diff --git a/docs/img/map.svg b/docs/img/map.svg
new file mode 100644
index 0000000..28fffed
--- /dev/null
+++ b/docs/img/map.svg
@@ -0,0 +1,295 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="143.05"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="New document 124">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.49497475"
+     inkscape:cx="-815.60294"
+     inkscape:cy="64.274519"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="570"
+     inkscape:window-x="1400"
+     inkscape:window-y="609"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-242.76955,-215.47765)">
+    <g
+       id="g35210"
+       transform="translate(2485.7696,-422.90351)">
+      <g
+         id="g33877">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2-8"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="676.76147" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2-6"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="643.2868" />
+        <rect
+           style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25"
+           id="rect2994-5-8-9"
+           width="57.498154"
+           height="136.54524"
+           x="-1922.3376"
+           y="641.63354" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-7-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="647.34906" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-6-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="647.34906" />
+        <rect
+           style="opacity:0.95999995;fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-7-1"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-3-9"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-5-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="716.60101" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-4-0"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="716.60101" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-5-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1913.9596"
+           y="752.37817" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-7-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1893.5885"
+           y="752.37817" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000041, 4.50000041;stroke-dashoffset:0"
+           id="rect2994-3-8-0-9-2"
+           width="57.498154"
+           height="28.746929"
+           x="-1821.1605"
+           y="712.60596" />
+        <rect
+           style="opacity:0.95999995;fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-7-2-6-4"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="716.75659" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-4-1-5-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="716.75659" />
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.50000024;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.50000043, 4.50000043;stroke-dashoffset:0"
+           id="rect2994-3-3-8-3-4"
+           width="57.498154"
+           height="29.100485"
+           x="-1821.1605"
+           y="747.47668" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-8-1-8-3"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="751.4505" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-5-0-3-5"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="751.4505" />
+        <rect
+           style="opacity:0.95999995;fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-7-8-9-5"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="647.349" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-8-9-9-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="647.349" />
+        <rect
+           style="opacity:0.95999995;fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-7-6-7-8"
+           width="20.371059"
+           height="20.622461"
+           x="-1812.7825"
+           y="680.82367" />
+        <rect
+           style="opacity:0.95999995;fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:1.50000012;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-5-4-1-6"
+           width="20.371059"
+           height="20.622461"
+           x="-1792.4115"
+           y="680.82367" />
+        <path
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+           d="m -1873.2175,657.6603 51.9673,0"
+           id="path4066-2-4-6"
+           sodipodi:nodetypes="cc"
+           inkscape:connector-curvature="0" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-3"
+           d="m -1873.2175,691.16137 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-4"
+           d="m -1873.2175,726.93859 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+        <path
+           inkscape:connector-curvature="0"
+           sodipodi:nodetypes="cc"
+           id="path4066-2-4-6-6"
+           d="m -1873.2175,762.71582 51.9674,0"
+           style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.50000024;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      </g>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/operator.svg
----------------------------------------------------------------------
diff --git a/docs/img/operator.svg b/docs/img/operator.svg
new file mode 100644
index 0000000..88379ec
--- /dev/null
+++ b/docs/img/operator.svg
@@ -0,0 +1,241 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="170"
+   id="svg6946"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="operator.svg">
+  <defs
+     id="defs6948">
+    <marker
+       inkscape:stockid="Arrow1Mend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mend"
+       style="overflow:visible">
+      <path
+         id="path3864"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.4,0,0,-0.4,-4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="2.5452459"
+     inkscape:cx="388.74049"
+     inkscape:cy="101.23661"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata6951">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-127.12556,-140.3642)">
+    <g
+       id="g8234"
+       transform="translate(181.30051,-1.8278624)">
+      <g
+         id="g7569">
+        <rect
+           style="fill:#a7dbd8;fill-opacity:1;stroke:#333333;stroke-width:1.99999976;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect7007"
+           width="271.42856"
+           height="134.01833"
+           x="206.42857"
+           y="145.01993" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="338.51025"
+           y="270.93359"
+           id="text7535"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             x="338.51025"
+             y="270.93359"
+             id="tspan7539">Operator Function</tspan></text>
+      </g>
+      <g
+         transform="translate(2.7779195,-6.8185297)"
+         id="g7564">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect7007-0"
+           width="169.28571"
+           height="88.571434"
+           x="293.57141"
+           y="163.32396" />
+        <text
+           xml:space="preserve"
+           style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+           x="299.37442"
+           y="244.45761"
+           id="text7535-3"
+           sodipodi:linespacing="125%"><tspan
+             sodipodi:role="line"
+             x="299.37442"
+             y="244.45761"
+             id="tspan7539-2">User-defined Function</tspan></text>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         id="rect7760"
+         d="m 158.27381,149.96659 0,25.28125 -26.71577,0 0,73.5625 26.71577,0 0,25.28125 36.08334,-62.0625 -36.08334,-62.0625 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="162.55525"
+         y="287.18451"
+         id="text3845-0-1"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2"
+           x="162.55525"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="162.55525"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170">Input Data</tspan></text>
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2"
+         d="m 247.45907,159.58539 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2-3"
+         d="m 247.45907,185.29972 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <path
+         sodipodi:nodetypes="cccccccc"
+         inkscape:connector-curvature="0"
+         id="rect7760-2-2"
+         d="m 247.45907,211.01397 0,5.24456 -26.71577,0 0,6.85017 26.71577,0 0,4.88742 36.08334,-8.49108 z"
+         style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0" />
+      <g
+         transform="translate(8.8900712,-1.6668884)"
+         id="g8092">
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 344.01749,180.66459 c -1.00984,0 -2.00307,0.0799 -2.97054,0.2303 l 0,2.95759 c -1.30309,0.241 -2.55224,0.63153 -3.72535,1.16364 l -1.4731,-2.53334 c -1.55054,0.73183 -2.98661,1.67422 -4.2732,2.77576 l 1.49745,2.56971 c -0.89425,0.81963 -1.6956,1.73909 -2.38618,2.7394 l -2.54444,-1.46667 c -0.94054,1.39727 -1.69549,2.93147 -2.24008,4.55759 l 2.60531,1.49092 c -0.30383,1.03528 -0.50752,2.11403 -0.59654,3.22425 l -2.93402,0 c -0.029,0.42867 -0.0487,0.86099 -0.0487,1.29698 0,1.29482 0.11742,2.55972 0.36523,3.78183 l 3.00706,0 c 0.23267,0.96251 0.55193,1.88931 0.9496,2.77576 l -2.54444,1.46667 c 0.72632,1.55023 1.66379,2.97813 2.76358,4.26668 l 2.56879,-1.47879 c 0.61288,0.67281 1.27837,1.29285 1.9966,1.85456 l -1.4731,2.53333 c 1.35328,1.01142 2.84542,1.85684 4.44364,2.48486 l 1.48527,-2.55758 c 0.82162,0.29731 1.67903,0.52861 2.55662,0.69091 l 0,2.95759 c 0.96746,0.1504 1.96069,0.2303 2.97054,0.2303 0.72232,0 1.42903,-0.0434 2.13051,-0.12122 l 0,-2.93333 c 0.92198,-0.12
 099 1.82433,-0.31291 2.69053,-0.58183 l 1.49746,2.56971 c 1.62727,-0.56836 3.1491,-1.35393 4.54103,-2.31516 l -1.46092,-2.53334 c 0.8221,-0.58602 1.59164,-1.23797 2.28878,-1.96364 l 2.58097,1.47879 c 1.13091,-1.26838 2.10046,-2.6809 2.86096,-4.21819 l -2.53226,-1.45456 c 0.47863,-0.99754 0.85459,-2.05339 1.12004,-3.15152 l 2.99488,0 c 0.24782,-1.22211 0.37742,-2.48701 0.37742,-3.78183 0,-0.43599 -0.0318,-0.86831 -0.0609,-1.29698 l -2.92185,0 c -0.10054,-1.25362 -0.34579,-2.46806 -0.71829,-3.62425 l 2.58096,-1.47879 c -0.58053,-1.61591 -1.37593,-3.12911 -2.34965,-4.50911 l -2.53227,1.45456 c -0.77844,-1.06674 -1.67797,-2.03832 -2.69053,-2.88487 l 1.4731,-2.54546 c -1.32833,-1.05198 -2.80059,-1.9333 -4.38277,-2.60607 l -1.4731,2.54546 c -1.22625,-0.48946 -2.53039,-0.82848 -3.88362,-1.00607 l 0,-2.93333 c -0.70148,-0.0778 -1.40819,-0.12122 -2.13051,-0.12122 z m 0,13.5758 c 3.01222,0 5.45411,2.43125 5.45411,5.43033 0,2.99908 -2.44189,5.43032 -5.45411,5.43032 -3.01223,0 -5.45411,-2.43124
  -5.45411,-5.43032 0,-2.99908 2.44188,-5.43033 5.45411,-5.43033 z"
+           id="path7903-09"
+           inkscape:connector-curvature="0" />
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 369.14084,167.18601 c -0.63817,0 -1.26583,0.0505 -1.87721,0.14554 l 0,1.86903 c -0.82348,0.15229 -1.61288,0.39908 -2.35421,0.73534 l -0.9309,-1.60091 c -0.97987,0.46246 -1.88737,1.058 -2.70042,1.75411 l 0.94629,1.62391 c -0.56511,0.51796 -1.0715,1.099 -1.50792,1.73114 l -1.60794,-0.92685 c -0.59436,0.88299 -1.07144,1.8525 -1.4156,2.88013 l 1.64641,0.94217 c -0.192,0.65424 -0.32073,1.33593 -0.37697,2.03754 l -1.85414,0 c -0.0184,0.27088 -0.0308,0.54409 -0.0308,0.81961 0,0.81825 0.0742,1.61759 0.23081,2.38989 l 1.90028,0 c 0.14703,0.60825 0.34879,1.19393 0.60009,1.75412 l -1.60793,0.92685 c 0.45899,0.97966 1.05142,1.88199 1.74642,2.6963 l 1.62332,-0.93452 c 0.3873,0.42517 0.80786,0.81701 1.26174,1.17197 l -0.93092,1.60092 c 0.85519,0.63916 1.79814,1.17341 2.80812,1.57029 l 0.9386,-1.61625 c 0.51923,0.18789 1.06105,0.33405 1.61564,0.43662 l 0,1.86902 c 0.61137,0.0951 1.23904,0.14554 1.87721,0.14554 0.45645,0 0.90306,-0.0274 1.34635,-0.0765 l 0,-1.8537 c 0.58263,-0.0765 
 1.15287,-0.19773 1.70027,-0.36768 l 0.94629,1.62391 c 1.02834,-0.35918 1.99006,-0.85561 2.86967,-1.46305 l -0.92321,-1.60092 c 0.51952,-0.37033 1.00582,-0.78233 1.44637,-1.24091 l 1.63102,0.93451 c 0.71468,-0.80155 1.32738,-1.69418 1.80797,-2.66565 l -1.60025,-0.9192 c 0.30247,-0.63038 0.54006,-1.29762 0.7078,-1.99157 l 1.89258,0 c 0.15661,-0.7723 0.23852,-1.57165 0.23852,-2.3899 0,-0.27552 -0.0202,-0.54872 -0.0385,-0.81961 l -1.84643,0 c -0.0635,-0.79221 -0.21852,-1.55967 -0.45392,-2.29031 l 1.63102,-0.93452 c -0.36686,-1.02115 -0.86951,-1.97741 -1.48484,-2.84948 l -1.60025,0.91919 c -0.49193,-0.6741 -1.06037,-1.2881 -1.70026,-1.82307 l 0.93092,-1.60858 c -0.83943,-0.66479 -1.76981,-1.22172 -2.76966,-1.64689 l -0.93092,1.60859 c -0.77491,-0.30932 -1.59905,-0.52354 -2.45421,-0.63577 l 0,-1.8537 c -0.44329,-0.0491 -0.88989,-0.0765 -1.34635,-0.0765 z m 0,8.57911 c 1.90355,0 3.44667,1.5364 3.44667,3.43165 0,1.89525 -1.54312,3.43164 -3.44667,3.43164 -1.90355,0 -3.44669,-1.53639 -3.44669
 ,-3.43164 0,-1.89525 1.54314,-3.43165 3.44669,-3.43165 z"
+           id="path7903-0-4"
+           inkscape:connector-curvature="0" />
+        <path
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:2;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 399.41472,172.32967 c -1.00984,0 -2.00307,0.0799 -2.97054,0.2303 l 0,2.95759 c -1.30309,0.24099 -2.55225,0.63153 -3.72536,1.16364 l -1.47309,-2.53335 c -1.55054,0.73184 -2.98661,1.67423 -4.2732,2.77578 l 1.49745,2.5697 c -0.89425,0.81963 -1.69559,1.73908 -2.38618,2.73941 l -2.54444,-1.46668 c -0.94053,1.39727 -1.6955,2.93146 -2.24008,4.55759 l 2.60531,1.49091 c -0.30383,1.03529 -0.50751,2.11404 -0.59655,3.22426 l -2.93401,0 c -0.029,0.42866 -0.0487,0.86098 -0.0487,1.29697 0,1.29484 0.11742,2.55973 0.36523,3.78183 l 3.00707,0 c 0.23266,0.96252 0.55192,1.88932 0.94959,2.77577 l -2.54444,1.46666 c 0.72632,1.55025 1.66379,2.97814 2.76359,4.26669 l 2.56879,-1.47879 c 0.61288,0.6728 1.27836,1.29285 1.99659,1.85456 l -1.4731,2.53333 c 1.35327,1.01143 2.84542,1.85684 4.44364,2.48486 l 1.48526,-2.55758 c 0.82163,0.29731 1.67904,0.52861 2.55663,0.6909 l 0,2.95759 c 0.96745,0.15041 1.96069,0.2303 2.97054,0.2303 0.72232,0 1.42904,-0.0434 2.13051,-0.1212 l 0,-2.93335 c 0.92198,-0
 .12099 1.82433,-0.3129 2.69054,-0.58182 l 1.49744,2.56971 c 1.62728,-0.56836 3.14911,-1.35393 4.54104,-2.31516 l -1.46093,-2.53335 c 0.82211,-0.58602 1.59165,-1.23796 2.28879,-1.96364 l 2.58096,1.47879 c 1.13093,-1.26838 2.10047,-2.6809 2.86098,-4.21819 l -2.53227,-1.45456 c 0.47863,-0.99753 0.85459,-2.05338 1.12004,-3.15152 l 2.99489,0 c 0.2478,-1.2221 0.37741,-2.48699 0.37741,-3.78183 0,-0.43599 -0.0318,-0.86831 -0.0609,-1.29697 l -2.92185,0 c -0.10053,-1.25363 -0.34578,-2.46807 -0.71829,-3.62426 l 2.58097,-1.47879 c -0.58052,-1.61591 -1.37593,-3.12911 -2.34966,-4.5091 l -2.53226,1.45455 c -0.77844,-1.06673 -1.67798,-2.03832 -2.69053,-2.88486 l 1.4731,-2.54546 c -1.32833,-1.05199 -2.80059,-1.9333 -4.38277,-2.60608 l -1.4731,2.54547 c -1.22626,-0.48947 -2.53039,-0.82848 -3.88363,-1.00607 l 0,-2.93334 c -0.70147,-0.0778 -1.40818,-0.12121 -2.1305,-0.12121 z m 0,13.5758 c 3.01223,0 5.45412,2.43124 5.45412,5.43032 0,2.99909 -2.44189,5.43033 -5.45412,5.43033 -3.01222,0 -5.45411,-2.43124
  -5.45411,-5.43033 0,-2.99908 2.44189,-5.43032 5.45411,-5.43032 z"
+           id="path7903-6-1"
+           inkscape:connector-curvature="0" />
+      </g>
+      <g
+         transform="translate(367.1568,8.8134766e-7)"
+         id="g8115-1">
+        <path
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:1.99999988000000006;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           d="m 150.27381,149.96659 0,25.28125 -26.71577,0 0,73.5625 26.71577,0 0,25.28125 36.08334,-62.0625 -36.08334,-62.0625 z"
+           id="rect7760-0"
+           inkscape:connector-curvature="0" />
+      </g>
+      <text
+         xml:space="preserve"
+         style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="251.40457"
+         y="247.67683"
+         id="text7535-1"
+         sodipodi:linespacing="125%"><tspan
+           style="font-style:italic;text-align:center;text-anchor:middle;-inkscape-font-specification:Arial Italic"
+           sodipodi:role="line"
+           x="251.40457"
+           y="247.67683"
+           id="tspan7539-3">UDF</tspan><tspan
+           style="font-style:italic;text-align:center;text-anchor:middle;-inkscape-font-specification:Arial Italic"
+           id="tspan8168"
+           sodipodi:role="line"
+           x="251.40457"
+           y="267.67682">Calls</tspan></text>
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="522.07141"
+         y="287.18451"
+         id="text3845-0-1-8"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2-4"
+           x="522.07141"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="522.07141"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170-7">Output Data</tspan></text>
+      <text
+         xml:space="preserve"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         x="341.65457"
+         y="287.18451"
+         id="text3845-0-1-8-1"
+         sodipodi:linespacing="125%"><tspan
+           sodipodi:role="line"
+           id="tspan3847-6-2-4-5"
+           x="341.65457"
+           y="287.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle" /><tspan
+           sodipodi:role="line"
+           x="341.65457"
+           y="307.18451"
+           style="font-size:16px;text-align:center;text-anchor:middle"
+           id="tspan8170-7-1">Operator</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/recorddm.svg
----------------------------------------------------------------------
diff --git a/docs/img/recorddm.svg b/docs/img/recorddm.svg
new file mode 100644
index 0000000..bcc20e0
--- /dev/null
+++ b/docs/img/recorddm.svg
@@ -0,0 +1,263 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="125.61"
+   id="svg2"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="recorddm.svg">
+  <defs
+     id="defs4">
+    <marker
+       inkscape:stockid="Arrow1Mend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mend"
+       style="overflow:visible">
+      <path
+         id="path3864"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(-0.4,0,0,-0.4,-4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path3861"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.4,0,0,0.4,4,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Lstart"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow1Lstart"
+       style="overflow:visible">
+      <path
+         id="path3855"
+         d="M 0,0 5,-5 -12.5,0 5,5 0,0 z"
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1pt"
+         transform="matrix(0.8,0,0,0.8,10,0)"
+         inkscape:connector-curvature="0" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.69753138"
+     inkscape:cx="-137.00168"
+     inkscape:cy="180.66416"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="986"
+     inkscape:window-x="1400"
+     inkscape:window-y="193"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata7">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-330.7807,-543.08649)">
+    <flowRoot
+       xml:space="preserve"
+       id="flowRoot4781"
+       style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       transform="translate(92.933548,154.52058)"><flowRegion
+         id="flowRegion4783"><rect
+           id="rect4785"
+           width="326.42029"
+           height="111.51003"
+           x="256.47308"
+           y="449.82095" /></flowRegion><flowPara
+         id="flowPara4789" /></flowRoot>    <g
+       id="g6802"
+       transform="translate(269.2258,-3.4995098)">
+      <g
+         transform="matrix(0.35294118,0,0,0.35294118,328.10303,357.09655)"
+         id="g4847">
+        <rect
+           style="fill:#eeeeee;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-3-1"
+           width="482.55975"
+           height="82"
+           x="10.420063"
+           y="649.36505" />
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5"
+           width="77.441345"
+           height="78"
+           x="12.920331"
+           y="651.04889" />
+        <rect
+           style="fill:#eeeeee;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-3"
+           width="77.441345"
+           height="78"
+           x="93.032051"
+           y="651.04889" />
+        <rect
+           style="fill:#b4a7d6;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-7"
+           width="77.441345"
+           height="78"
+           x="173.1438"
+           y="651.04889" />
+        <rect
+           style="fill:#666666;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-8"
+           width="77.441345"
+           height="78"
+           x="253.25552"
+           y="651.04889" />
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-4"
+           width="77.441345"
+           height="78"
+           x="333.36725"
+           y="651.04889" />
+        <rect
+           style="fill:#e1f5c4;fill-opacity:1;stroke:#333333;stroke-width:5.66666651;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:0"
+           id="rect2985-5-0"
+           width="77.441345"
+           height="78"
+           x="413.47897"
+           y="651.04889" />
+      </g>
+      <text
+         sodipodi:linespacing="125%"
+         id="text3845"
+         y="649.5304"
+         x="441.89203"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="649.5304"
+           x="441.89203"
+           id="tspan3847"
+           sodipodi:role="line">Field: Type</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text3845-0"
+         y="561.53864"
+         x="391.07892"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="561.53864"
+           x="391.07892"
+           id="tspan3847-6"
+           sodipodi:role="line">Record</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4753"
+         y="639.59271"
+         x="553.21973"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           id="tspan4757"
+           y="639.59271"
+           x="553.21973"
+           sodipodi:role="line">Value</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4799"
+         y="658.17999"
+         x="540.8418"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:12px"
+           y="658.17999"
+           x="540.8418"
+           id="tspan4801"
+           sodipodi:role="line">●</tspan></text>
+      <text
+         sodipodi:linespacing="125%"
+         id="text4799-8"
+         y="637.34369"
+         x="540.8418"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:12px"
+           y="637.34369"
+           x="540.8418"
+           id="tspan4801-8"
+           sodipodi:role="line">●</tspan></text>
+      <text
+         inkscape:transform-center-y="66.906019"
+         inkscape:transform-center-x="196.66315"
+         sodipodi:linespacing="125%"
+         id="text4827"
+         y="660.42902"
+         x="552.11816"
+         style="font-size:48px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           style="font-size:16px"
+           y="660.42902"
+           x="552.11816"
+           id="tspan4829"
+           sodipodi:role="line">Key</tspan></text>
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878"
+         d="m 331.78518,578.49146 0,-3.70254 80.12761,0 5.47415,-4.53532 5.40619,4.53532 79.29829,0 0,3.62459"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878-7"
+         d="m 446.54075,621.05807 0,3.77371 8.224,0 5.14272,4.62249 5.22951,-4.62249 8.86137,0 0,-3.69427"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+      <path
+         sodipodi:nodetypes="ccccccc"
+         inkscape:connector-curvature="0"
+         id="path4878-7-9"
+         d="m 537.48048,623.51239 -4.794,0 0,15.62417 -5.87227,6.39539 5.87227,6.50333 0,15.66121 4.69308,0"
+         style="fill:none;stroke:#000000;stroke-width:2;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/reduce.svg
----------------------------------------------------------------------
diff --git a/docs/img/reduce.svg b/docs/img/reduce.svg
new file mode 100644
index 0000000..e0e8df3
--- /dev/null
+++ b/docs/img/reduce.svg
@@ -0,0 +1,425 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="185"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="reduce.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-6"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-1"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.4"
+     inkscape:cx="233.75455"
+     inkscape:cy="86.830647"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-492.58981,-171.02527)">
+    <g
+       id="g44557"
+       transform="translate(284.86629,-3.7883928)">
+      <g
+         transform="translate(2158.7696,-421.90351)"
+         id="g35174">
+        <g
+           id="g33519"
+           transform="translate(-475,10)">
+          <g
+             transform="matrix(1,0,0,1.0123433,-1388.1056,2483.203)"
+             id="g7121-1"
+             style="opacity:0.95999995">
+            <rect
+               style="fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25"
+               id="rect2994-5-8-90"
+               width="57.498154"
+               height="134.88037"
+               x="267.76794"
+               y="-1829.9814" />
+            <g
+               id="g3792-4-7-6"
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1148.6406)"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-0-7-5"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-5-6-7"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1115.5741)"
+               id="g3792-8-9-8-25"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-46-7-5"
+                 style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-9-3-0"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1080.2331)"
+               id="g3792-0-2-2-3"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-4-2-5-5"
+                 style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-4-4-4-8"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,190.00545,-1044.8921)"
+               id="g3792-8-4-7-3-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-4-75-5-7"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-7-4-7-36"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248249, 4.47248249;stroke-dashoffset:0"
+               id="rect2994-3-8-0-9-22"
+               width="57.498154"
+               height="28.256905"
+               x="368.94507"
+               y="-1759.9644" />
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1080.0794)"
+               id="g3792-8-5-0-5-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-7-2-6-3"
+                 style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-4-1-5-2"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248249, 4.47248249;stroke-dashoffset:0"
+               id="rect2994-3-3-8-3-7"
+               width="57.498154"
+               height="27.763"
+               x="368.94507"
+               y="-1725.4464" />
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1045.8085)"
+               id="g3792-0-1-1-5-0"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-4-8-1-8-8"
+                 style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-4-5-0-3-7"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <rect
+               style="fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49082756;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.47248248, 4.47248248;stroke-dashoffset:0"
+               id="rect2994-3-36-6-8"
+               width="57.498154"
+               height="60.043583"
+               x="368.94507"
+               y="-1828.2587" />
+            <g
+               id="g3792-1-2-1-9"
+               transform="matrix(0.4074212,0,0,0.40742032,291.0657,-1147.8529)"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1660.405"
+                 x="211.71547"
+                 height="50"
+                 width="50"
+                 id="rect2992-7-8-9-3"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1660.405"
+                 x="261.71548"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-8-9-9-4"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <g
+               transform="matrix(0.4074212,0,0,0.40742032,291.18262,-1115.5741)"
+               id="g3792-8-4-9-0-1-4"
+               style="stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+              <rect
+                 y="-1658.4716"
+                 x="211.42857"
+                 height="50"
+                 width="50"
+                 id="rect2992-1-4-7-6-7-4"
+                 style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+              <rect
+                 y="-1658.4716"
+                 x="261.42859"
+                 height="50"
+                 width="50"
+                 id="rect2992-8-0-7-5-4-1-3"
+                 style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65918398;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+            </g>
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1814.1501 51.96735,0"
+               id="path4066-2-4-62"
+               sodipodi:nodetypes="cc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1781.0836 31.9416,21.7791 c 3.17115,-4.7193 8.17328,-0.6496 5.3385,3.64 l 14.77695,10.0756"
+               id="path4066-8-5-0-1"
+               sodipodi:nodetypes="cccc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1710.4016 51.94148,-70.682"
+               id="path4066-0-6-2-3"
+               sodipodi:nodetypes="cc"
+               inkscape:connector-curvature="0" />
+            <path
+               style="fill:none;stroke:#000000;stroke-width:1.49082756;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+               d="m 316.88805,-1745.7426 14.775,9.772 c 3.00165,-5.0886 9.07942,-0.9487 5.48925,3.6306 l 31.78455,21.022"
+               id="path4066-8-5-2-5-4"
+               sodipodi:nodetypes="cccc"
+               inkscape:connector-curvature="0" />
+          </g>
+        </g>
+      </g>
+      <path
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)"
+         d="m 559.17478,185.71313 22.75265,-0.035 -0.12654,35.13393"
+         id="path4066-2-0-9-4-1-2"
+         sodipodi:nodetypes="ccc"
+         inkscape:connector-curvature="0" />
+      <text
+         sodipodi:linespacing="125%"
+         id="text39591"
+         y="190.05518"
+         x="491.41794"
+         style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+         xml:space="preserve"><tspan
+           y="190.05518"
+           x="491.41794"
+           id="tspan39593"
+           sodipodi:role="line">Key Field</tspan></text>
+    </g>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/spargel_example.png
----------------------------------------------------------------------
diff --git a/docs/img/spargel_example.png b/docs/img/spargel_example.png
new file mode 100644
index 0000000..21d20f8
Binary files /dev/null and b/docs/img/spargel_example.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/spargel_example_input.png
----------------------------------------------------------------------
diff --git a/docs/img/spargel_example_input.png b/docs/img/spargel_example_input.png
new file mode 100644
index 0000000..01ead46
Binary files /dev/null and b/docs/img/spargel_example_input.png differ

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
new file mode 100644
index 0000000..d26e081
--- /dev/null
+++ b/docs/index.md
@@ -0,0 +1,11 @@
+---
+layout: docs
+title: Apache Flink Documentation
+---
+
+Lorem ipsum dolor sit amet, consectetur adipisicing elit, sed do eiusmod
+tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam,
+quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo
+consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse
+cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non
+proident, sunt in culpa qui officia deserunt mollit anim id est laborum.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/iterations.md
----------------------------------------------------------------------
diff --git a/docs/iterations.md b/docs/iterations.md
new file mode 100644
index 0000000..063f121
--- /dev/null
+++ b/docs/iterations.md
@@ -0,0 +1,188 @@
+---
+title:  "Iterations"
+---
+
+Iterative algorithms occur in many domains of data analysis, such as *machine learning* or *graph analysis*. Such algorithms are crucial in order to realize the promise of Big Data to extract meaningful information out of your data. With increasing interest to run these kinds of algorithms on very large data sets, there is a need to execute iterations in a massively parallel fashion.
+
+Stratosphere programs implement iterative algorithms by defining a **step function** and embedding it into a special iteration operator. There are two  variants of this operator: **Iterate** and **Delta Iterate**. Both operators repeatedly invoke the step function on the current iteration state until a certain termination condition is reached.
+
+Here, we provide background on both operator variants and outline their usage. The [programming guides]({{ site.baseurl }}/docs/0.4/programming_guides/) explain how to implement the operators in both [Scala]({{ site.baseurl }}/docs/0.4/programming_guides/scala.html) and [Java]({{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations). We also provide a **vertex-centric graph processing API** called [Spargel]({{ site.baseurl }}/docs/0.4/programming_guides/spargel.html).
+
+The following table provides an overview of both operators:
+
+<table class="table table-striped table-hover table-bordered">
+	<thead>
+		<th></th>
+		<th class="text-center">Iterate</th>
+		<th class="text-center">Delta Iterate</th>
+	</thead>
+	<tr>
+		<td class="text-center" width="20%"><strong>Iteration Input</strong></td>
+		<td class="text-center" width="40%"><strong>Partial Solution</strong></td>
+		<td class="text-center" width="40%"><strong>Workset</strong> and <strong>Solution Set</strong></td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Step Function</strong></td>
+		<td colspan="2" class="text-center">Arbitrary Data Flows</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>State Update</strong></td>
+		<td class="text-center">Next <strong>partial solution</strong></td>
+		<td>
+			<ul>
+				<li>Next workset</li>
+				<li><strong>Changes to solution set</strong></li>
+			</ul>
+		</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Iteration Result</strong></td>
+		<td class="text-center">Last partial solution</td>
+		<td class="text-center">Solution set state after last iteration</td>
+	</tr>
+	<tr>
+		<td class="text-center"><strong>Termination</strong></td>
+		<td>
+			<ul>
+				<li><strong>Maximum number of iterations</strong> (default)</li>
+				<li>Custom aggregator convergence</li>
+			</ul>
+		</td>
+		<td>
+			<ul>
+				<li><strong>Maximum number of iterations or empty workset</strong> (default)</li>
+				<li>Custom aggregator convergence</li>
+			</ul>
+		</td>
+	</tr>
+</table>
+
+<section id="iterate">
+Iterate Operator
+----------------
+
+The **iterate operator** covers the *simple form of iterations*: in each iteration, the **step function** consumes the **entire input** (the *result of the previous iteration*, or the *initial data set*), and computes the **next version of the partial solution** (e.g. `map`, `reduce`, `join`, etc.).
+
+<p class="text-center">
+    <img alt="Iterate Operator" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_iterate_operator.png" />
+</p>
+
+  1. **Iteration Input**: Initial input for the *first iteration* from a *data source* or *previous operators*.
+  2. **Step Function**: The step function will be executed in each iteration. It is an arbitrary data flow consisting of operators like `map`, `reduce`, `join`, etc. (see [programming model]({{ site.baseurl }}/docs/0.4/programming_guides/pmodel.html) for details) and depends on your specific task at hand.
+  3. **Next Partial Solution**: In each iteration, the output of the step function will be fed back into the *next iteration*.
+  4. **Iteration Result**: Output of the *last iteration* is written to a *data sink* or used as input to the *following operators*.
+
+There are multiple options to specify **termination conditions** for an iteration:
+
+  - **Maximum number of iterations**: Without any further conditions, the iteration will be executed this many times.
+  - **Custom aggregator convergence**: Iterations allow to specify *custom aggregators* and *convergence criteria* like sum aggregate the number of emitted records (aggregator) and terminate if this number is zero (convergence criterion).
+
+You can also think about the iterate operator in pseudo-code:
+
+{% highlight java %}
+IterationState state = getInitialState();
+
+while (!terminationCriterion()) {
+	state = step(state);
+}
+
+setFinalState(state);
+{% endhighlight %}
+
+<div class="panel panel-default">
+	<div class="panel-body">
+	See the <strong><a href="{{ site.baseurl }}/docs/0.4/programming_guides/scala.html">Scala</a> and <a href="{{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations">Java</a> programming guides</strong> for details and code examples.</div>
+</div>
+
+### Example: Incrementing Numbers
+
+In the following example, we **iteratively incremenet a set numbers**:
+
+<p class="text-center">
+    <img alt="Iterate Operator Example" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_iterate_operator_example.png" />
+</p>
+
+  1. **Iteration Input**: The inital input is read from a data source and consists of five single-field records (integers `1` to `5`).
+  2. **Step function**: The step function is a single `map` operator, which increments the integer field from `i` to `i+1`. It will be applied to every record of the input.
+  3. **Next Partial Solution**: The output of the step function will be the output of the map operator, i.e. records with incremented integers.
+  4. **Iteration Result**: After ten iterations, the initial numbers will have been incremented ten times, resulting in integers `11` to `15`.
+
+```
+// 1st           2nd                       10th
+map(1) -> 2      map(2) -> 3      ...      map(10) -> 11
+map(2) -> 3      map(3) -> 4      ...      map(11) -> 12
+map(3) -> 4      map(4) -> 5      ...      map(12) -> 13
+map(4) -> 5      map(5) -> 6      ...      map(13) -> 14
+map(5) -> 6      map(6) -> 7      ...      map(14) -> 15
+```
+
+Note that **1**, **2**, and **4** can be arbitrary data flows.
+</section>
+
+<section id="delta_iterate">
+Delta Iterate Operator
+----------------------
+
+The **delta iterate operator** covers the case of **incremental iterations**. Incremental iterations **selectively modify elements** of their **solution solution** and evolve the solution rather than fully recompute it.
+
+Where applicable, this leads to **more efficient algorithms**, because not every element in the solution set changes in each iteration. This allows to **focus on the hot parts** of the solution and leave the **cold parts untouched**. Frequently, the majority of the solution cools down comparatively fast and the later iterations operate only on a small subset of the data.
+
+<p class="text-center">
+    <img alt="Delta Iterate Operator" width="60%" src="{{ site.baseurl }}/docs/0.4/img/iterations_delta_iterate_operator.png" />
+</p>
+
+  1. **Iteration Input**: The initial workset and solution set are read from *data sources* or *previous operators* as input to the first iteration.
+  2. **Step Function**: The step function will be executed in each iteration. It is an arbitrary data flow consisting of operators like `map`, `reduce`, `join`, etc. (see [programming model]({{ site.baseurl }}/docs/0.4/programming_guides/pmodel.html) for details) and depends on your specific task at hand.
+  3. **Next Workset/Update Solution Set**: The *next workset* drives the iterative computation and will be fed back into the *next iteration*. Furthermore, the solution set will be updated and implicitly forwarded (it is not required to be rebuild). Both data sets can be updated by different operators of the step function.
+  4. **Iteration Result**: After the *last iteration*, the *solution set* is written to a *data sink* or used as input to the *following operators*.
+
+The default **termination condition** for delta iterations is specified by the **empty workset convergence criterion** and a **maximum number of iterations**. The iteration will terminate when a produced *next workset* is empty or when the maximum number of iterations is reached. It is also possible to specify a **custom aggregator** and **convergence criterion**.
+
+You can also think about the iterate operator in pseudo-code:
+{% highlight java %}
+IterationState workset = getInitialState();
+IterationState solution = getInitialSolution();
+
+while (!terminationCriterion()) {
+	(delta, workset) = step(workset, solution);
+
+	solution.update(delta)
+}
+
+setFinalState(solution);
+{% endhighlight %}
+
+<div class="panel panel-default">
+	<div class="panel-body">
+	See the <strong><a href="{{ site.baseurl }}/docs/0.4/programming_guides/scala.html">Scala</a> and <a href="{{ site.baseurl }}/docs/0.4/programming_guides/java.html#iterations">Java</a> programming guides</strong> for details and code examples.</div>
+</div>
+
+### Example: Propagate Minimum in Graph
+
+In the following example, every vertex has an **ID** and a **coloring**. Each vertex will propagete its vertex ID to neighboring vertices. The **goal** is to *assign the minimum ID to every vertex in a subgraph*. If a received ID is smaller then the current one, it changes to the color of the vertex with the received ID. One application of this can be found in *community analysis* or *connected components* computation.
+
+<p class="text-center">
+    <img alt="Delta Iterate Operator Example" width="100%" src="{{ site.baseurl }}/docs/0.4/img/iterations_delta_iterate_operator_example.png" />
+</p>
+
+The **intial input** is set as **both workset and solution set.** In the above figure, the colors visualize the **evolution of the solution set**. With each iteration, the color of the minimum ID is spreading in the respective subgraph. At the same time, the amount of work (exchanged and compared vertex IDs) decreases with each iteration. This corresponds to the **decreasing size of the workset**, which goes from all seven vertices to zero after three iterations, at which time the iteration terminates. The **important observation** is that *the lower subgraph converges before the upper half* does and the delta iteration is able to capture this with the workset abstraction.
+
+In the upper subgraph **ID 1** (*orange*) is the **minimum ID**. In the **first iteration**, it will get propagated to vertex 2, which will subsequently change its color to orange. Vertices 3 and 4 will receive **ID 2** (in *yellow*) as their current minimum ID and change to yellow. Because the color of *vertex 1* didn't change in the first iteration, it can be skipped it in the next workset.
+
+In the lower subgraph **ID 5** (*cyan*) is the **minimum ID**. All vertices of the lower subgraph will receive it in the first iteration. Again, we can skip the unchanged vertices (*vertex 5*) for the next workset.
+
+In the **2nd iteration**, the workset size has already decreased from seven to five elements (vertices 2, 3, 4, 6, and 7). These are part of the iteration and further propagate their current minimum IDs. After this iteration, the lower subgraph has already converged (**cold part** of the graph), as it has no elements in the workset, whereas the upper half needs a further iteration (**hot part** of the graph) for the two remaining workset elements (vertices 3 and 4).
+
+The iteration **terminates**, when the workset is empty after the **3rd iteration**.
+</section>
+
+<section id="supersteps">
+Superstep Synchronization
+-------------------------
+
+We referred to each execution of the step function of an iteration operator as *a single iteration*. In parallel setups, **multiple instances of the step function are evaluated in parallel** on different partitions of the iteration state. In many settings, one evaluation of the step function on all parallel instances forms a so called **superstep**, which is also the granularity of synchronization. Therefore, *all* parallel tasks of an iteration need to complete the superstep, before a next superstep will be initialized. **Termination criteria** will also be evaluated at superstep barriers.
+
+<p class="text-center">
+    <img alt="Supersteps" width="50%" src="{{ site.baseurl }}/docs/0.4/img/iterations_supersteps.png" />
+</p>
+</section>
\ No newline at end of file


[6/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/cogroup.svg
----------------------------------------------------------------------
diff --git a/docs/img/cogroup.svg b/docs/img/cogroup.svg
new file mode 100644
index 0000000..26ec868
--- /dev/null
+++ b/docs/img/cogroup.svg
@@ -0,0 +1,856 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="cogroup.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow1Lstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Lstart"
+       style="overflow:visible">
+      <path
+         id="path3855"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.8) translate(12.5,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="EmptyTriangleInM"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="EmptyTriangleInM"
+       style="overflow:visible">
+      <path
+         id="path4009"
+         d="M 5.77,0.0 L -2.88,5.0 L -2.88,-5.0 L 5.77,0.0 z "
+         style="fill-rule:evenodd;fill:#FFFFFF;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(-0.4) translate(-4.5,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow2Lstart"
+       style="overflow:visible">
+      <path
+         id="path3873"
+         style="fill-rule:evenodd;stroke-width:0.62500000;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 L -2.2072895,0.016013256 L 8.7185884,-4.0017078 C 6.9730900,-1.6296469 6.9831476,1.6157441 8.7185878,4.0337352 z "
+         transform="scale(1.1) translate(1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Mstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Mstart"
+       style="overflow:visible">
+      <path
+         id="path3861"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.4) translate(10,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow1Sstart"
+       orient="auto"
+       refY="0.0"
+       refX="0.0"
+       id="Arrow1Sstart"
+       style="overflow:visible">
+      <path
+         id="path3867"
+         d="M 0.0,0.0 L 5.0,-5.0 L -12.5,0.0 L 5.0,5.0 L 0.0,0.0 z "
+         style="fill-rule:evenodd;stroke:#000000;stroke-width:1.0pt"
+         transform="scale(0.2) translate(6,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-77"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35951"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35953"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35955"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35957"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35959"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35961"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35963"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35965"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35967"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35969"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-93"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-0"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36515"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36517"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36519"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36521"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36523"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36525"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36527"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36529"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker36531"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path36533"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-36"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-73"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="0.49497475"
+     inkscape:cx="-657.23669"
+     inkscape:cy="-312.51371"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="1161"
+     inkscape:window-x="1400"
+     inkscape:window-y="18"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g34970"
+       transform="translate(1902.9756,-921.79345)">
+      <rect
+         y="1225.0306"
+         x="-1230.1406"
+         height="136.06552"
+         width="57.281994"
+         id="rect2994-5-9-9-1"
+         style="opacity:0.95999995;fill:#f3f3f3;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1915.8924)"
+         id="g3792-4-1-79-5">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-93-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-2-4"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-4-4"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1950.654)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-3-2"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-7-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-1-0"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,2020.1768)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-2-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-2-0"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-0-8"
+         transform="matrix(0.40773951,0,0,0.41277147,-1308.0944,1985.4152)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-2-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-1-7"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="1173.7782"
+         x="-1152.9976"
+         height="32.716614"
+         width="232.52985"
+         id="rect2994-5-9-4-7-4"
+         style="opacity:0.95999995;fill:#f3f3f3;stroke:#000000;stroke-width:1.5;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-2-4"
+         d="m -1181.1127,1241.642 148.6025,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="ccccc"
+         id="path4066-2-0-5-7-0"
+         d="m -1181.1127,1276.4035 19.9637,14.4355 c 5.7719,-7.3309 13.2528,-0.6632 8.304,6.0044 l 19.8863,14.3795 219.2601,-0.059"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="ccc"
+         id="path4066-2-0-7-0-9"
+         d="m -1181.1127,1311.1647 48.4175,-34.3028 100.4399,-0.4584"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-3-5"
+         d="m -1181.1127,1345.9263 28.844,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-4-1"
+         d="m -1068.1581,1200.5444 -0.4619,128.2519"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-6-0"
+         d="m -948.6859,1200.4497 0.052,24.4359"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40773951,0,0,0.41277147,-1055.2752,1864.387)"
+         id="g3792-4-1-1-7-1">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-5-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65632725;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-1-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-7-0"
+         transform="matrix(0.40271073,0,0,0.40768065,-1173.5759,1856.0714)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.70198441;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-4-9"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.70198441;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-1-3"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="1297.5363"
+         x="-909.70618"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-4-6"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <g
+         id="g34054-2"
+         transform="translate(261.32072,614.33529)">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1252.1926,1369.3045)"
+           id="g3792-8-5-02-5-8"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-9-0"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-7-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1199.1018,1369.3045)"
+           id="g3792-0-2-0-2-3-3-5-0"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none" />
+      </g>
+      <g
+         transform="translate(-1.0606602,2.4748737)"
+         id="g34960">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0"
+           id="rect2994-3-0-0-6-7"
+           width="103.5"
+           height="28.5"
+           x="-1146.2062"
+           y="1329.1628" />
+        <g
+           transform="translate(24.82072,575.33529)"
+           id="g34070-5">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-1-17-4-6-1"
+             transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-8-4-1-8-1"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-5-3-2-1-5"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             id="g3792-0-2-0-2-3-7-5-1"
+             transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)">
+            <rect
+               style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-4-2-4-5-3-9-5-91"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-4-4-8-4-7-60-6-0"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+      <g
+         transform="translate(-400.5,92.5)"
+         id="g34945">
+        <rect
+           style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999976;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.4999996, 4.4999996;stroke-dashoffset:0"
+           id="rect2994-3-0-0-1-4"
+           width="103.5"
+           height="62.617905"
+           x="-628.79382"
+           y="1135.4185" />
+        <g
+           transform="translate(426.32072,486.33529)"
+           id="g34078-4">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-1">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-10"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-5"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-15" />
+        </g>
+        <g
+           transform="translate(426.32073,519.81003)"
+           id="g34078-4-5">
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+             id="g3792-1-9-8-4-1-2">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-7-3-3-6-10-4"
+               width="50"
+               height="50"
+               x="211.71547"
+               y="-1660.405" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-8-6-83-2-5-6"
+               width="50"
+               height="50"
+               x="261.71548"
+               y="-1660.405" />
+          </g>
+          <g
+             style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+             transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+             id="g3792-4-1-1-4-3-15-4">
+            <rect
+               style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-0-0-9-1-0-6-1"
+               width="50"
+               height="50"
+               x="211.42857"
+               y="-1658.4716" />
+            <rect
+               style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+               id="rect2992-8-5-7-6-0-2-3-9"
+               width="50"
+               height="50"
+               x="261.42859"
+               y="-1658.4716" />
+          </g>
+        </g>
+      </g>
+    </g>
+    <path
+       inkscape:connector-curvature="0"
+       sodipodi:nodetypes="ccc"
+       id="path4066-2-0-9-4-1-2"
+       d="m 669.67823,271.03445 22.75265,-0.035 -0.12654,35.13393"
+       style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.5;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    <text
+       xml:space="preserve"
+       style="font-size:16px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;line-height:125%;letter-spacing:0px;word-spacing:0px;fill:#000000;fill-opacity:1;stroke:none;font-family:Arial;-inkscape-font-specification:Arial"
+       x="601.92139"
+       y="275.3765"
+       id="text39591"
+       sodipodi:linespacing="125%"><tspan
+         sodipodi:role="line"
+         id="tspan39593"
+         x="601.92139"
+         y="275.3765">Key Field</tspan></text>
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/img/cross.svg
----------------------------------------------------------------------
diff --git a/docs/img/cross.svg b/docs/img/cross.svg
new file mode 100644
index 0000000..5161f39
--- /dev/null
+++ b/docs/img/cross.svg
@@ -0,0 +1,893 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="800"
+   height="195"
+   id="svg35289"
+   version="1.1"
+   inkscape:version="0.48.3.1 r9886"
+   sodipodi:docname="match.svg">
+  <defs
+     id="defs35291">
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35250"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35252"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35254"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35256"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35258"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35260"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-0"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-7"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35403"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35405"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35407"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35409"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35411"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35413"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-9"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-75"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35572"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35574"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35576"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35578"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35580"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35582"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35584"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35586"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35588"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35590"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="Arrow2Lend-3"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path4456-77"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35951"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35953"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35955"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35957"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35959"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35961"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35963"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35965"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+    <marker
+       inkscape:stockid="Arrow2Lend"
+       orient="auto"
+       refY="0"
+       refX="0"
+       id="marker35967"
+       style="overflow:visible">
+      <path
+         inkscape:connector-curvature="0"
+         id="path35969"
+         style="font-size:12px;fill-rule:evenodd;stroke-width:0.625;stroke-linejoin:round"
+         d="M 8.7185878,4.0337352 -2.2072895,0.01601326 8.7185884,-4.0017078 c -1.7454984,2.3720609 -1.7354408,5.6174519 -6e-7,8.035443 z"
+         transform="matrix(-1.1,0,0,-1.1,-1.1,0)" />
+    </marker>
+  </defs>
+  <sodipodi:namedview
+     id="base"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1.0"
+     inkscape:pageopacity="0.0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.979899"
+     inkscape:cx="318.74501"
+     inkscape:cy="39.054602"
+     inkscape:document-units="px"
+     inkscape:current-layer="layer1"
+     showgrid="false"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:window-width="1916"
+     inkscape:window-height="570"
+     inkscape:window-x="1400"
+     inkscape:window-y="609"
+     inkscape:window-maximized="0" />
+  <metadata
+     id="metadata35294">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(-484.80217,-248.14373)">
+    <g
+       id="g35082"
+       transform="translate(1984.0346,-344.20004)">
+      <rect
+         y="753.82758"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-8"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="683.20093"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-4"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="719.13367"
+         x="-1171.0267"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-9"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="753.82758"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-6"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="649.7262"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-5"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="683.20093"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-1"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="719.13367"
+         x="-1055.1145"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0-3"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <rect
+         y="595.75568"
+         x="-1175.5856"
+         height="31.868748"
+         width="227.71063"
+         id="rect2994-5-9-4-9"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <rect
+         y="647.87219"
+         x="-1250.59"
+         height="136.05968"
+         width="57.377071"
+         id="rect2994-5-9-7"
+         style="opacity:0.95999995;fill:#f3f3f3;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;stroke-dashoffset:25" />
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40795244,0,0,0.41238689,-1328.5519,1337.5985)"
+         id="g3792-4-1-0">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-5"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-13"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-9-5-2"
+         transform="matrix(0.40688131,0,0,0.41130412,-1328.2718,1370.405)">
+        <rect
+           style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-46-8-99"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-9-7-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-6"
+         transform="matrix(0.40475762,0,0,0.40915735,-1327.7167,1436.4245)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-0"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-6"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-8-4-7-0-4"
+         transform="matrix(0.40581951,0,0,0.41023078,-1327.9943,1403.4839)">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-1-4-75-4-3"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-0-7-4-2-05"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none"
+         transform="matrix(0.40946483,0,0,0.41232107,-1082.4563,1285.2048)"
+         id="g3792-4-1-1-41">
+        <rect
+           style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65060806;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-0-0-9-45"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65060806;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-5-7-6-8"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <g
+         style="opacity:0.95999995;stroke-width:1.53030467;stroke-miterlimit:4;stroke-dasharray:none"
+         id="g3792-0-2-0-2-5"
+         transform="matrix(0.40135661,0,0,0.4041563,-1197.5462,1271.8679)">
+        <rect
+           style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.72435784;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-4-2-4-5-1"
+           width="50"
+           height="50"
+           x="211.42857"
+           y="-1658.4716" />
+        <rect
+           style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.72435784;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none"
+           id="rect2992-8-4-4-8-4-9"
+           width="50"
+           height="50"
+           x="261.42859"
+           y="-1658.4716" />
+      </g>
+      <rect
+         y="649.7262"
+         x="-1171.0269"
+         height="28.5"
+         width="103.5"
+         id="rect2994-3-0-0"
+         style="opacity:0.95999995;fill:#dddddd;fill-opacity:1;stroke:#000000;stroke-width:1.49999988;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:4.49999962, 4.49999962;stroke-dashoffset:0" />
+      <g
+         id="g34046">
+        <g
+           id="g3792-1-9-6"
+           transform="matrix(0.40795244,0,0,0.41238689,-1252.5361,1338.3958)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-7"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-9"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40795244,0,0,0.41238689,-1199.4354,1337.5985)"
+           id="g3792-0-2-0-2-3-9"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-61"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-8"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34054">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1252.1926,1369.3045)"
+           id="g3792-8-5-02-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-9"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-7"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1199.1018,1369.3045)"
+           id="g3792-0-2-0-2-3-3-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-2-9"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-6-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34062">
+        <g
+           id="g3792-1-9-9-9"
+           transform="matrix(0.40581951,0,0,0.41023078,-1252.0845,1404.2772)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-2-2"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-8-9"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40581951,0,0,0.41023078,-1198.7711,1403.4839)"
+           id="g3792-0-2-0-2-3-2-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-6-1"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-5-7"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34070">
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1251.7436,1436.4245)"
+           id="g3792-0-1-17-4-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-8-4-1-8"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-5-3-2-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1198.4405,1436.4245)"
+           id="g3792-0-2-0-2-3-7-5"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-2-4-5-3-9-5"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-4-8-4-7-60-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34078">
+        <g
+           id="g3792-1-9-8-4"
+           transform="matrix(0.40795244,0,0,0.41238689,-1138.1239,1338.3958)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-3-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-83-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-4-3"
+           transform="matrix(0.40795244,0,0,0.41238689,-1082.0231,1337.5985)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-1-0"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-0-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.65707684;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34086">
+        <g
+           transform="matrix(0.40688131,0,0,0.41130412,-1137.7804,1369.3045)"
+           id="g3792-8-5-02-8-6"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-1-7-9-0-7"
+             style="fill:#f3de4a;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-0-4-4-4-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-8-59"
+           transform="matrix(0.40688131,0,0,0.41130412,-1081.6895,1369.3045)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-7-8"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-08-2"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.66670442;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34094">
+        <g
+           id="g3792-1-9-9-7-0"
+           transform="matrix(0.40581951,0,0,0.41023078,-1137.6723,1404.2772)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1660.405"
+             x="211.71547"
+             height="50"
+             width="50"
+             id="rect2992-7-3-2-9-78"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1660.405"
+             x="261.71548"
+             height="50"
+             width="50"
+             id="rect2992-8-8-6-8-0-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-2-38"
+           transform="matrix(0.40581951,0,0,0.41023078,-1081.3588,1403.4839)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-4-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-7-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.67629814;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <g
+         id="g34102">
+        <g
+           transform="matrix(0.40475762,0,0,0.40915735,-1137.3314,1436.4245)"
+           id="g3792-0-1-17-4-8-1"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-4-8-4-1-9-8"
+             style="fill:#f3b33f;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-4-5-3-2-0-6"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+        <g
+           id="g3792-4-1-1-3-6"
+           transform="matrix(0.40475762,0,0,0.40915735,-1081.0281,1436.4245)"
+           style="opacity:0.95999995;stroke-width:1.50000167;stroke-miterlimit:4;stroke-dasharray:none">
+          <rect
+             y="-1658.4716"
+             x="211.42857"
+             height="50"
+             width="50"
+             id="rect2992-0-0-9-9-6"
+             style="fill:#4ecdc4;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+          <rect
+             y="-1658.4716"
+             x="261.42859"
+             height="50"
+             width="50"
+             id="rect2992-8-5-7-6-2-1"
+             style="fill:#e0e4cc;fill-opacity:1;stroke:#000000;stroke-width:3.68594265;stroke-linecap:round;stroke-linejoin:round;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+        </g>
+      </g>
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-34"
+         d="m -1201.5038,663.97626 25.9157,-0.11419"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-5-0"
+         d="m -1201.5574,698.55143 25.9693,0"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-7-9"
+         d="m -1201.6105,733.38363 26.0224,-0.14274"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-3-6"
+         d="m -1201.6636,768.07765 26.0755,-0.1474"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-8"
+         d="m -1092.6535,621.72237 -0.1005,25.32033"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+      <path
+         inkscape:connector-curvature="0"
+         sodipodi:nodetypes="cc"
+         id="path4066-2-0-9-5-2"
+         d="m -975.21023,621.80923 0,23.97522"
+         style="opacity:0.95999995;fill:none;stroke:#000000;stroke-width:1.49999988;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none;marker-start:none;marker-mid:none;marker-end:url(#Arrow2Lend)" />
+    </g>
+  </g>
+</svg>


[2/7] [FLINK-962] Initial import of documentation from website into source code (closes #34)

Posted by uc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/java_api_quickstart.md b/docs/java_api_quickstart.md
new file mode 100644
index 0000000..75f4c7c
--- /dev/null
+++ b/docs/java_api_quickstart.md
@@ -0,0 +1,126 @@
+---
+title: "Quickstart: Java API"
+---
+
+<p class="lead">Start working on your Stratosphere Java program in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">The only requirements are working <strong>Maven 3.0.4</strong> (or higher) and <strong>Java 6.x</strong> (or higher) installations.</p>
+</section>
+
+<section id="create_project">
+  <div class="page-header"><h2>Create Project</h2></div>
+
+  <p class="lead">Use one of the following commands to <strong>create a project</strong>:</p>
+
+  <ul class="nav nav-tabs" style="border-bottom: none;">
+      <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
+      <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
+  </ul>
+  <div class="tab-content">
+      <div class="tab-pane active" id="quickstart-script">
+{% highlight bash %}
+$ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart.sh | bash
+{% endhighlight %}
+      </div>
+      <div class="tab-pane" id="maven-archetype">
+{% highlight bash %}
+$ mvn archetype:generate                             \
+    -DarchetypeGroupId=eu.stratosphere               \
+    -DarchetypeArtifactId=quickstart-java            \
+    -DarchetypeVersion={{site.current_stable}}
+{% endhighlight %}
+      This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
+      </div>
+  </div>
+</section>
+
+<section id="inspect_project">
+  <div class="page-header"><h2>Inspect Project</h2></div>
+  <p class="lead">There will be a <strong>new directory in your working directory</strong>. If you've used the <em>curl</em> approach, the directory is called <code>quickstart</code>. Otherwise, it has the name of your artifactId.</p>
+  <p class="lead">The sample project is a <strong>Maven project</strong>, which contains two classes. <em>Job</em> is a basic skeleton program and <em>WordCountJob</em> a working example. Please note that the <em>main</em> method of both classes allow you to start Stratosphere in a development/testing mode.</p>
+  <p class="lead">We recommend to <strong>import this project into your IDE</strong> to develop and test it. If you use Eclipse, the <a href="http://www.eclipse.org/m2e/">m2e plugin</a> allows to <a href="http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import">import Maven projects</a>. Some Eclipse bundles include that plugin by default, other require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box.</p>
+</section>
+
+<section id="build_project">
+<div class="alert alert-danger">A note to Mac OS X users: The default JVM heapsize for Java is too small for Stratosphere. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.</div>
+  <div class="page-header"><h2>Build Project</h2></div>
+  <p class="lead">If you want to <strong>build your project</strong>, go to your project directory and issue the <code>mvn clean package</code> command. You will <strong>find a jar</strong> that runs on every Stratosphere cluster in <code>target/stratosphere-project-0.1-SNAPSHOT.jar</code>.</p>
+</section>
+
+<section id="next_steps">
+  <div class="page-header"><h2>Next Steps</h2></div>
+  <p class="lead"><strong>Write your application!</strong></p>
+  <p>The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occurs in all Wikipedia texts.</p>
+ <br>
+<b>Sample Input:</b> <br>
+{% highlight bash %}
+big data is big
+{% endhighlight %}
+<b>Sample Output:</b> <br>
+{% highlight bash %}
+big 2
+data 1
+is 1
+{% endhighlight %}
+
+<p>The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and writes the prints the resulting words and counts to std-out.</p>
+
+{% highlight java %}
+public class WordCount {
+  
+  public static void main(String[] args) throws Exception {
+    
+    // set up the execution environment
+    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+    
+    // get input data
+    DataSet<String> text = env.fromElements(
+        "To be, or not to be,--that is the question:--",
+        "Whether 'tis nobler in the mind to suffer",
+        "The slings and arrows of outrageous fortune",
+        "Or to take arms against a sea of troubles,"
+        );
+    
+    DataSet<Tuple2<String, Integer>> counts = 
+        // split up the lines in pairs (2-tuples) containing: (word,1)
+        text.flatMap(new LineSplitter())
+        // group by the tuple field "0" and sum up tuple field "1"
+        .groupBy(0)
+        .aggregate(Aggregations.SUM, 1);
+
+    // emit result
+    counts.print();
+    
+    // execute program
+    env.execute("WordCount Example");
+  }
+}
+{% endhighlight %}
+
+<p>The operations are defined by specialized classes, here the LineSplitter class.</p>
+
+{% highlight java %}
+public class LineSplitter extends FlatMapFunction<String, Tuple2<String, Integer>> {
+
+  @Override
+  public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+    // normalize and split the line into words
+    String[] tokens = value.toLowerCase().split("\\W+");
+    
+    // emit the pairs
+    for (String token : tokens) {
+      if (token.length() > 0) {
+        out.collect(new Tuple2<String, Integer>(token, 1));
+      }
+    }
+  }
+}
+
+{% endhighlight %}
+
+<p><a href="https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-java-examples/src/main/java/eu/stratosphere/example/java/wordcount/WordCount.java">Check GitHub</a> for the full example code.</p>
+
+<p class="lead">For a complete overview over our Java API, have a look at the <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/java.html">Stratosphere Documentation</a> and <a href="{{ site.baseurl }}/docs/{{site.current_stable_documentation}}/programming_guides/examples_java.html">further example programs</a>. If you have any trouble, ask on our <a href="https://groups.google.com/forum/#!forum/stratosphere-dev">Mailing list</a>. We are happy to provide help.</p>
+</section>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/local_execution.md
----------------------------------------------------------------------
diff --git a/docs/local_execution.md b/docs/local_execution.md
new file mode 100644
index 0000000..cd60f62
--- /dev/null
+++ b/docs/local_execution.md
@@ -0,0 +1,106 @@
+---
+title:  "Local Execution"
+---
+
+# Local Execution/Debugging
+
+Stratosphere can run on a single machine, even in a single Java Virtual Machine. This allows users to test and debug Stratosphere programs locally. This section gives an overview of the local execution mechanisms.
+
+**NOTE:** Please also refer to the [debugging section]({{site.baseurl}}/docs/0.5/programming_guides/java.html#debugging) in the Java API documentation for a guide to testing and local debugging utilities in the Java API.
+
+The local environments and executors allow you to run Stratosphere programs in local Java Virtual Machine, or with within any JVM as part of existing programs. Most examples can be launched locally by simply hitting the "Run" button of your IDE.
+
+If you are running Stratosphere programs locally, you can also debug your program like any other Java program. You can either use `System.out.println()` to write out some internal variables or you can use the debugger. It is possible to set breakpoints within `map()`, `reduce()` and all the other methods.
+
+The `JobExecutionResult` object, which is returned after the execution finished, contains the program runtime and the accumulator results.
+
+*Note:* The local execution environments do not start any web frontend to monitor the execution.
+
+
+# Maven Dependency
+
+If you are developing your program in a Maven project, you have to add the `stratosphere-clients` module using this dependency:
+
+```xml
+<dependency>
+  <groupId>eu.stratosphere</groupId>
+  <artifactId>stratosphere-clients</artifactId>
+  <version>{{site.current_stable}}</version>
+</dependency>
+```
+
+# Local Environment
+
+The `LocalEnvironment` is a handle to local execution for Stratosphere programs. Use it to run a program within a local JVM - standalone or embedded in other programs.
+
+The local environment is instantiated via the method `ExecutionEnvironment.createLocalEnvironment()`. By default, it will use as many local threads for execution as your machine has CPU cores (hardware contexts). You can alternatively specify the desired parallelism. The local environment can be configured to log to the console using `enableLogging()`/`disableLogging()`.
+
+In most cases, calling `ExecutionEnvironment.getExecutionEnvironment()` is the even better way to go. That method returns a `LocalEnvironment` when the program is started locally (outside the command line interface), and it returns a pre-configured environment for cluster execution, when the program is invoked by the [command line interface]({{ site.baseurl }}/docs/0.5/program_execution/cli_client.html).
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+
+    DataSet<String> data = env.readTextFile("file:///path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("file:///path/to/result");
+
+    env.execute();
+}
+```
+
+
+# Local Executor
+
+The *LocalExecutor* is similar to the local environment, but it takes a *Plan* object, which describes the program as a single executable unit. The *LocalExecutor* is typically used with the Scala API. 
+
+The following code shows how you would use the `LocalExecutor` with the Wordcount example for Scala Programs:
+
+```scala
+public static void main(String[] args) throws Exception {
+    val input = TextFile("hdfs://path/to/file")
+
+    val words = input flatMap { _.toLowerCase().split("""\W+""") filter { _ != "" } }
+    val counts = words groupBy { x => x } count()
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+  
+    val plan = new ScalaPlan(Seq(output), "Word Count")
+    LocalExecutor.executePlan(p);
+}
+```
+
+
+# LocalDistributedExecutor
+
+Stratosphere also offers a `LocalDistributedExecutor` which starts multiple TaskManagers within one JVM. The standard `LocalExecutor` starts one JobManager and one TaskManager in one JVM.
+With the `LocalDistributedExecutor` you can define the number of TaskManagers to start. This is useful for debugging network related code and more of a developer tool than a user tool.
+
+```java
+public static void main(String[] args) throws Exception {
+    ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+    DataSet<String> data = env.readTextFile("hdfs://path/to/file");
+
+    data
+        .filter(new FilterFunction<String>() {
+            public boolean filter(String value) {
+                return value.startsWith("http://");
+            }
+        })
+        .writeAsText("hdfs://path/to/result");
+
+    Plan p = env.createProgramPlan();
+    LocalDistributedExecutor lde = new LocalDistributedExecutor();
+    lde.startNephele(2); // start two TaskManagers
+    lde.run(p);
+}
+```
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/local_setup.md
----------------------------------------------------------------------
diff --git a/docs/local_setup.md b/docs/local_setup.md
new file mode 100644
index 0000000..b49118a
--- /dev/null
+++ b/docs/local_setup.md
@@ -0,0 +1,108 @@
+---
+title:  "Local Setup"
+---
+
+This documentation is intended to provide instructions on how to run Stratosphere locally on a single machine.
+
+# Download
+
+Go to the [downloads page]({{site.baseurl}}/downloads/) and get the ready to run package. If you want to interact with Hadoop (e.g. HDFS or HBase), make sure to pick the Stratosphere package **matching your Hadoop version**. When in doubt or you plan to just work with the local file system pick the package for Hadoop 1.2.x.
+
+# Requirements
+
+Stratosphere runs on **Linux**, **Mac OS X** and **Windows**. The only requirement for a local setup is **Java 1.6.x** or higher. The following manual assumes a *UNIX-like environment*, for Windows see [Stratosphere on Windows](#windows).
+
+You can check the correct installation of Java by issuing the following command:
+
+```bash
+java -version
+```
+
+The command should output something comparable to the following:
+
+```bash
+java version "1.6.0_22"
+Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
+Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)
+```
+
+# Configuration
+
+**For local mode Stratosphere is ready to go out of the box and you don't need to change the default configuration.**
+
+The out of the box configuration will use your default Java installation. You can manually set the environment variable `JAVA_HOME` or the configuration key `env.java.home` in `conf/stratosphere-conf.yaml` if you want to manually override the Java runtime to use. Consult the [configuration page]({{site.baseurl}}/docs/0.4/setup/config.html) for further details about configuring Stratosphere.
+
+# Starting Stratosphere
+
+**You are now ready to start Stratosphere.** Unpack the downloaded archive and change to the newly created `stratosphere` directory. There you can start Stratosphere in local mode:
+
+```bash
+$ tar xzf stratosphere-*.tgz
+$ cd stratosphere
+$ bin/start-local.sh
+Starting job manager
+```
+
+You can check that the system is running by checking the log files in the `logs` directory:
+
+```bash
+$ tail log/stratosphere-*-jobmanager-*.log
+INFO ... - Initializing memory manager with 409 megabytes of memory
+INFO ... - Trying to load eu.stratosphere.nephele.jobmanager.scheduler.local.LocalScheduler as scheduler
+INFO ... - Setting up web info server, using web-root directory ...
+INFO ... - Web info server will display information about nephele job-manager on localhost, port 8081.
+INFO ... - Starting web info server for JobManager on port 8081
+```
+
+The JobManager will also start a web frontend on port 8081, which you can check with your browser at `http://localhost:8081`.
+
+# Stratosphere on Windows
+
+If you want to run Stratosphere on Windows you need to download, unpack and configure the Stratosphere archive as mentioned above. After that you can either use the **Windows Batch** file (`.bat`) or use **Cygwin**  to run the Stratosphere Jobmanager.
+
+To start Stratosphere in local mode from the *Windows Batch*, open the command window, navigate to the `bin/` directory of Stratosphere and run `start-local.bat`.
+
+```bash
+$ cd stratosphere
+$ cd bin
+$ start-local.bat
+Starting Stratosphere job manager. Webinterface by default on http://localhost:8081/.
+Do not close this batch window. Stop job manager by pressing Ctrl+C.
+```
+
+After that, you need to open a second terminal to run jobs using `stratosphere.bat`.
+
+With *Cygwin* you need to start the Cygwin Terminal, navigate to your Stratosphere directory and run the `start-local.sh` script:
+
+```bash
+$ cd stratosphere
+$ bin/start-local.sh
+Starting Nephele job manager
+```
+
+If you are installing Stratosphere from the git repository and you are using the Windows git shell, Cygwin can produce a failure similiar to this one:
+
+```bash
+c:/stratosphere/bin/start-local.sh: line 30: $'\r': command not found
+```
+
+This error occurs, because git is automatically transforming UNIX line endings to Windows style line endings when running in Windows. The problem is, that Cygwin can only deal with UNIX style line endings. The solution is to adjust the Cygwin settings to deal with the correct line endings by following these three steps:
+
+1. Start a Cygwin shell.
+
+2. Determine your home directory by entering
+
+```bash
+cd; pwd
+```
+
+It will return a path under the Cygwin root path.
+
+2.  Using NotePad, WordPad or a different text editor open the file `.bash_profile` in the home directory and append the following: (If the file does not exist you have to create it)
+
+```bash
+export SHELLOPTS
+set -o igncr
+```
+
+Save the file and open a new bash shell.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/quickstart/plotPoints.py
----------------------------------------------------------------------
diff --git a/docs/quickstart/plotPoints.py b/docs/quickstart/plotPoints.py
new file mode 100755
index 0000000..fa04d31
--- /dev/null
+++ b/docs/quickstart/plotPoints.py
@@ -0,0 +1,82 @@
+#!/usr/bin/python
+import sys
+import matplotlib.pyplot as plt
+import csv
+import os
+
+if len(sys.argv) < 4 or not sys.argv[1] in ['points', 'result']:
+  print "Usage: plot-clusters.py (points|result) <src-file> <pdf-file-prefix>"
+  sys.exit(1)
+
+inFile = sys.argv[1]
+inFile = sys.argv[2]
+outFilePx = sys.argv[3]
+
+inFileName = os.path.splitext(os.path.basename(inFile))[0]
+outFile = os.path.join(".", outFilePx+"-plot.pdf")
+
+########### READ DATA
+
+cs = []
+xs = []
+ys = []
+
+minX = None
+maxX = None
+minY = None
+maxY = None
+
+if sys.argv[1] == 'points':
+
+  with open(inFile, 'rb') as file:
+    for line in file:
+      # parse data
+      csvData = line.strip().split(' ')
+
+      x = float(csvData[0])
+      y = float(csvData[1])
+
+      if not minX or minX > x:
+        minX = x
+      if not maxX or maxX < x:
+        maxX = x
+      if not minY or minY > y:
+        minY = y
+      if not maxY or maxY < y:
+        maxY = y
+
+      xs.append(x)
+      ys.append(y)
+
+    # plot data
+    plt.clf()
+    plt.scatter(xs, ys, s=25, c="#999999", edgecolors='None', alpha=1.0)
+    plt.ylim([minY,maxY])
+    plt.xlim([minX,maxX])
+
+elif sys.argv[1] == 'result':
+
+  with open(inFile, 'rb') as file:
+    for line in file:
+      # parse data
+      csvData = line.strip().split(' ')
+
+      c = int(csvData[0])
+      x = float(csvData[1])
+      y = float(csvData[2])
+
+      cs.append(c)
+      xs.append(x)
+      ys.append(y)
+
+    # plot data
+    plt.clf()
+    plt.scatter(xs, ys, s=25, c=cs, edgecolors='None', alpha=1.0)
+    plt.ylim([minY,maxY])
+    plt.xlim([minX,maxX])
+
+
+plt.savefig(outFile, dpi=600)
+print "\nPlotted file: %s" % outFile
+
+sys.exit(0)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/run_example_quickstart.md b/docs/run_example_quickstart.md
new file mode 100644
index 0000000..600e3fd
--- /dev/null
+++ b/docs/run_example_quickstart.md
@@ -0,0 +1,154 @@
+---
+title: "Quick Start: Run K-Means Example"
+---
+
+
+<p class="lead">
+	This guide will demonstrate Stratosphere's features by example. You will see how you can leverage Stratosphere's Iteration-feature to find clusters in a dataset using <a href="http://en.wikipedia.org/wiki/K-means_clustering">K-Means clustering</a>. 
+	On the way, you will see the compiler, the status interface and the result of the algorithm.
+</p>
+
+
+<section id="data">
+  <div class="page-header">
+  	<h2>Generate Input Data</h2>
+  </div>
+  <p>Stratosphere contains a data generator for K-Means.</p>
+  {% highlight bash %}
+# Download Stratosphere
+wget {{ site.current_stable_dl }}
+tar xzf stratosphere-*.tgz 
+cd stratosphere-*
+mkdir kmeans
+cd kmeans
+# run data generator
+java -cp  ../examples/stratosphere-java-examples-{{ site.current_stable }}-KMeans.jar eu.stratosphere.example.java.clustering.util.KMeansDataGenerator 500 10 0.08
+cp /tmp/points .
+cp /tmp/centers .
+  {% endhighlight %}
+The generator has the following arguments:
+{% highlight bash %}
+KMeansDataGenerator <numberOfDataPoints> <numberOfClusterCenters> [<relative stddev>] [<centroid range>] [<seed>]
+{% endhighlight %}
+The <i>relative standard deviation</i> is an interesting tuning parameter: it determines the closeness of the points to the centers.
+<p>The <code>kmeans/</code> directory should now contain two files: <code>centers</code> and <code>points</code>.</p>
+
+
+<h2>Review Input Data</h2>
+Use the <code>plotPoints.py</code> tool to review the result of the data generator. <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Download Python Script</a>
+{% highlight bash %}
+python2.7 plotPoints.py points points input
+{% endhighlight %}
+
+
+Note: You might have to install <a href="http://matplotlib.org/">matplotlib</a> (<code>python-matplotlib</code> package on Ubuntu) to use the Python script.
+
+
+The following overview presents the impact of the different standard deviations on the input data.
+<div class="row" style="padding-top:15px">
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans003.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;padding-bottom:2px">relative stddev = 0.08</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans008.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
+		<a data-lightbox="inputs" href="{{site.baseurl}}/img/quickstart-example/kmeans015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/kmeans015.png" /></a>
+	</div>
+</div>
+</section>
+
+<section id="run">
+ <div class="page-header">
+  	<h2>Run Clustering</h2>
+  </div>
+We are using the generated input data to run the clustering using a Stratosphere job.
+{% highlight bash %}
+# go to the Stratosphere-root directory
+cd stratosphere
+# start Stratosphere (use ./bin/start-cluster.sh if you're on a cluster)
+./bin/start-local.sh
+# Start Stratosphere web client
+./bin/start-webclient.sh
+{% endhighlight %}
+
+<h2>Review Stratosphere Compiler</h2>
+
+The Stratosphere webclient allows to submit Stratosphere programs using a graphical user interface.
+
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/run-webclient.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/run-webclient.png" /></a>
+	</div>
+	<div class="col-md-6">
+		1. <a href="http://localhost:8080/launch.html">Open webclient on localhost:8080</a> <br>
+		2. Upload the 
+{% highlight bash %}
+examples/stratosphere-java-examples-0.5-SNAPSHOT-KMeansIterative.jar
+{% endhighlight %} file.<br>
+		3. Select it in the left box to see how the operators in the plan are connected to each other. <br>
+		4. Enter the arguments in the lower left box:
+{% highlight bash %}
+file://<pathToGenerated>points file://<pathToGenerated>centers file://<pathToGenerated>result 10
+{% endhighlight %}
+For example:
+{% highlight bash %}
+file:///tmp/stratosphere/kmeans/points file:///tmp/stratosphere/kmeans/centers file:///tmp/stratosphere/kmeans/result 20
+{% endhighlight %}
+	</div>
+</div>
+<hr>
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/compiler-webclient-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/compiler-webclient-new.png" /></a>
+	</div>
+
+	<div class="col-md-6">
+		1. Press the <b>RunJob</b> to see the optimzer plan. <br>
+		2. Inspect the operators and see the properties (input sizes, cost estimation) determined by the optimizer.
+	</div>
+</div>
+<hr>
+<div class="row" style="padding-top:15px">
+	<div class="col-md-6">
+		<a data-lightbox="compiler" href="{{site.baseurl}}/img/quickstart-example/jobmanager-running-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/jobmanager-running-new.png" /></a>
+	</div>
+	<div class="col-md-6">
+		1. Press the <b>Continue</b> button to start executing the job. <br>
+		2. <a href="http://localhost:8080/launch.html">Open Stratosphere's monitoring interface</a> to see the job's progress.<br>
+		3. Once the job has finished, you can analyize the runtime of the individual operators.
+	</div>
+</div>
+</section>
+
+<section id="result">
+ <div class="page-header">
+  	<h2>Analyze the Result</h2>
+  </div>
+Use the <a href="{{site.baseurl}}/quickstart/example-data/plotPoints.py">Python Script</a> again to visualize the result
+
+{% highlight bash %}
+python2.7 plotPoints.py result result result-pdf
+{% endhighlight %}
+
+The following three pictures show the results for the sample input above. Play around with the parameters (number of iterations, number of clusters) to see how they affect the result.
+
+<div class="row" style="padding-top:15px">
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.03</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result003.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result003.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.08</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result008.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result008.png" /></a>
+	</div>
+	<div class="col-md-4">
+		<div class="text-center" style="font-weight:bold;">relative stddev = 0.15</div>
+		<a data-lightbox="results" href="{{site.baseurl}}/img/quickstart-example/result015.png" data-lightbox="example-1"><img class="img-responsive" src="{{site.baseurl}}/img/quickstart-example/result015.png" /></a>
+	</div>
+</div>
+
+</section>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_examples.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_examples.md b/docs/scala_api_examples.md
new file mode 100644
index 0000000..ac930b3
--- /dev/null
+++ b/docs/scala_api_examples.md
@@ -0,0 +1,195 @@
+---
+title:  "Scala API Examples"
+---
+
+The following example programs showcase different applications of Stratosphere from simple word counting to graph algorithms.
+The code samples illustrate the use of **[Stratosphere's Scala API]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/scala.html)**. 
+
+The full source code of the following and more examples can be found in the **[stratosphere-scala-examples](https://github.com/stratosphere/stratosphere/tree/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples)** module.
+
+# Word Count
+
+WordCount is the "Hello World" of Big Data processing systems. It computes the frequency of words in a text collection. The algorithm works in two steps: First, the texts are splits the text to individual words. Second, the words are grouped and counted.
+
+```scala
+// read input data
+val input = TextFile(textInput)
+
+// tokenize words
+val words = input.flatMap { _.split(" ") map { (_, 1) } }
+
+// count by word
+val counts = words.groupBy { case (word, _) => word }
+  .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
+
+val output = counts.write(wordsOutput, CsvOutputFormat()))
+```
+
+The [WordCount example](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/wordcount/WordCount.scala) implements the above described algorithm with input parameters: `<degree of parallelism>, <text input path>, <output path>`. As test data, any text file will do.
+
+# Page Rank
+
+The PageRank algorithm computes the "importance" of pages in a graph defined by links, which point from one pages to another page. It is an iterative graph algorithm, which means that it repeatedly applies the same computation. In each iteration, each page distributes its current rank over all its neighbors, and compute its new rank as a taxed sum of the ranks it received from its neighbors. The PageRank algorithm was popularized by the Google search engine which uses the importance of webpages to rank the results of search queries.
+
+In this simple example, PageRank is implemented with a [bulk iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations) and a fixed number of iterations.
+
+```scala
+// cases classes so we have named fields
+case class PageWithRank(pageId: Long, rank: Double)
+case class Edge(from: Long, to: Long, transitionProbability: Double)
+
+// constants for the page rank formula
+val dampening = 0.85
+val randomJump = (1.0 - dampening) / NUM_VERTICES
+val initialRank = 1.0 / NUM_VERTICES
+  
+// read inputs
+val pages = DataSource(verticesPath, CsvInputFormat[Long]())
+val edges = DataSource(edgesPath, CsvInputFormat[Edge]())
+
+// assign initial rank
+val pagesWithRank = pages map { p => PageWithRank(p, initialRank) }
+
+// the iterative computation
+def computeRank(ranks: DataSet[PageWithRank]) = {
+
+    // send rank to neighbors
+    val ranksForNeighbors = ranks join edges
+        where { _.pageId } isEqualTo { _.from }
+        map { (p, e) => (e.to, p.rank * e.transitionProbability) }
+    
+    // gather ranks per vertex and apply page rank formula
+    ranksForNeighbors .groupBy { case (node, rank) => node }
+                      .reduce { (a, b) => (a._1, a._2 + b._2) }
+                      .map {case (node, rank) => PageWithRank(node, rank * dampening + randomJump) }
+}
+
+// invoke iteratively
+val finalRanks = pagesWithRank.iterate(numIterations, computeRank)
+val output = finalRanks.write(outputPath, CsvOutputFormat())
+```
+
+
+
+The [PageRank program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/graph/PageRank.scala) implements the above example.
+It requires the following parameters to run: `<pages input path>, <link input path>, <output path>, <num pages>, <num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Pages represented as an (long) ID separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five pages with IDs 1, 2, 12, 42, and 63.
+- Links are represented as pairs of page IDs which are separated by space characters. Links are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).
+
+For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
+
+# Connected Components
+
+The Connected Components algorithm identifies parts of a larger graph which are connected by assigning all vertices in the same connected part the same component ID. Similar to PageRank, Connected Components is an iterative algorithm. In each step, each vertex propagates its current component ID to all its neighbors. A vertex accepts the component ID from a neighbor, if it is smaller than its own component ID.
+
+This implementation uses a [delta iteration]({{site.baseurl}}/docs/{{site.current_stable}}/programming_guides/java.html#iterations): Vertices that have not changed their component id do not participate in the next step. This yields much better performance, because the later iterations typically deal only with a few outlier vertices.
+
+```scala
+// define case classes
+case class VertexWithComponent(vertex: Long, componentId: Long)
+case class Edge(from: Long, to: Long)
+
+// get input data
+val vertices = DataSource(verticesPath, CsvInputFormat[Long]())
+val directedEdges = DataSource(edgesPath, CsvInputFormat[Edge]())
+
+// assign each vertex its own ID as component ID
+val initialComponents = vertices map { v => VertexWithComponent(v, v) }
+val undirectedEdges = directedEdges flatMap { e => Seq(e, Edge(e.to, e.from)) }
+
+def propagateComponent(s: DataSet[VertexWithComponent], ws: DataSet[VertexWithComponent]) = {
+  val allNeighbors = ws join undirectedEdges
+        where { _.vertex } isEqualTo { _.from }
+        map { (v, e) => VertexWithComponent(e.to, v.componentId ) }
+    
+    val minNeighbors = allNeighbors groupBy { _.vertex } reduceGroup { cs => cs minBy { _.componentId } }
+
+    // updated solution elements == new workset
+    val s1 = s join minNeighbors
+        where { _.vertex } isEqualTo { _.vertex }
+        flatMap { (curr, candidate) =>
+            if (candidate.componentId < curr.componentId) Some(candidate) else None
+        }
+
+  (s1, s1)
+}
+
+val components = initialComponents.iterateWithDelta(initialComponents, { _.vertex }, propagateComponent,
+                    maxIterations)
+val output = components.write(componentsOutput, CsvOutputFormat())
+```
+
+The [ConnectedComponents program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/graph/ConnectedComponents.scala) implements the above example. It requires the following parameters to run: `<vertex input path>, <edge input path>, <output path> <max num iterations>`.
+
+Input files are plain text files and must be formatted as follows:
+- Vertices represented as IDs and separated by new-line characters.
+    * For example `"1\n2\n12\n42\n63\n"` gives five vertices with (1), (2), (12), (42), and (63).
+- Edges are represented as pairs for vertex IDs which are separated by space characters. Edges are separated by new-line characters:
+    * For example `"1 2\n2 12\n1 12\n42 63\n"` gives four (undirected) links (1)-(2), (2)-(12), (1)-(12), and (42)-(63).
+
+# Relational Query
+
+The Relational Query example assumes two tables, one with `orders` and the other with `lineitems` as specified by the [TPC-H decision support benchmark](http://www.tpc.org/tpch/). TPC-H is a standard benchmark in the database industry. See below for instructions how to generate the input data.
+
+The example implements the following SQL query.
+
+```sql
+SELECT l_orderkey, o_shippriority, sum(l_extendedprice) as revenue
+    FROM orders, lineitem
+WHERE l_orderkey = o_orderkey
+    AND o_orderstatus = "F" 
+    AND YEAR(o_orderdate) > 1993
+    AND o_orderpriority LIKE "5%"
+GROUP BY l_orderkey, o_shippriority;
+```
+
+The Stratosphere Scala program, which implements the above query looks as follows.
+
+```scala
+// --- define some custom classes to address fields by name ---
+case class Order(orderId: Int, status: Char, date: String, orderPriority: String, shipPriority: Int)
+case class LineItem(orderId: Int, extendedPrice: Double)
+case class PrioritizedOrder(orderId: Int, shipPriority: Int, revenue: Double)
+
+val orders = DataSource(ordersInputPath, DelimitedInputFormat(parseOrder))
+val lineItem2600s = DataSource(lineItemsInput, DelimitedInputFormat(parseLineItem))
+
+val filteredOrders = orders filter { o => o.status == "F" && o.date.substring(0, 4).toInt > 1993 && o.orderPriority.startsWith("5") }
+
+val prioritizedItems = filteredOrders join lineItems
+    where { _.orderId } isEqualTo { _.orderId } // join on the orderIds
+    map { (o, li) => PrioritizedOrder(o.orderId, o.shipPriority, li.extendedPrice) }
+
+val prioritizedOrders = prioritizedItems
+    groupBy { pi => (pi.orderId, pi.shipPriority) } 
+    reduce { (po1, po2) => po1.copy(revenue = po1.revenue + po2.revenue) }
+
+val output = prioritizedOrders.write(ordersOutput, CsvOutputFormat(formatOutput))
+```
+
+The [Relational Query program](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-examples/stratosphere-scala-examples/src/main/scala/eu/stratosphere/examples/scala/relational/RelationalQuery.scala) implements the above query. It requires the following parameters to run: `<orders input path>, <lineitem input path>, <output path>, <degree of parallelism>`.
+
+The orders and lineitem files can be generated using the [TPC-H benchmark](http://www.tpc.org/tpch/) suite's data generator tool (DBGEN). 
+Take the following steps to generate arbitrary large input files for the provided Stratosphere programs:
+
+1.  Download and unpack DBGEN
+2.  Make a copy of *makefile.suite* called *Makefile* and perform the following changes:
+
+```bash
+DATABASE = DB2
+MACHINE  = LINUX
+WORKLOAD = TPCH
+CC       = gcc
+```
+
+1.  Build DBGEN using *make*
+2.  Generate lineitem and orders relations using dbgen. A scale factor
+    (-s) of 1 results in a generated data set with about 1 GB size.
+
+```bash
+./dbgen -T o -s 1
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_guide.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_guide.md b/docs/scala_api_guide.md
new file mode 100644
index 0000000..4b43938
--- /dev/null
+++ b/docs/scala_api_guide.md
@@ -0,0 +1,1008 @@
+---
+title: "Scala API Programming Guide"
+---
+
+
+Scala Programming Guide
+=======================
+
+This guide explains how to develop Stratosphere programs with the Scala
+programming interface. It assumes you are familiar with the general concepts of
+Stratosphere's [Programming Model](pmodel.html "Programming Model"). We
+recommend to learn about the basic concepts first, before continuing with the
+[Java](java.html "Java Programming Guide") or this Scala programming guide.
+
+Here we will look at the general structure of a Scala job. You will learn how to
+write data sources, data sinks, and operators to create data flows that can be
+executed using the Stratosphere system.
+
+Writing Scala jobs requires an understanding of Scala, there is excellent
+documentation available [here](http://scala-lang.org/documentation/). Most
+of the examples can be understood by someone with a good understanding
+of programming in general, though.
+
+<section id="intro-example">
+Word Count Example
+------------------
+
+To start, let's look at a Word Count job implemented in Scala. This program is
+very simple but it will give you a basic idea of what a Scala job looks like.
+
+```scala
+import eu.stratosphere.client.LocalExecutor
+
+import eu.stratosphere.api.scala._
+import eu.stratosphere.api.scala.operators._
+
+object WordCount {
+  def main(args: Array[String]) {
+    val input = TextFile(textInput)
+
+    val words = input.flatMap { _.split(" ") map { (_, 1) } }
+
+    val counts = words.groupBy { case (word, _) => word }
+      .reduce { (w1, w2) => (w1._1, w1._2 + w2._2) }
+
+    val output = counts.write(wordsOutput, CsvOutputFormat())
+    val plan = new ScalaPlan(Seq(output))
+
+    LocalExecutor.execute(plan)
+  }
+}
+``` 
+
+Same as any Stratosphere job a Scala job consists of one or several data
+sources, one or several data sinks and operators in between these that transform
+data. Together these parts are referred to as the data flow graph. It dictates
+the way data is passed when a job is executed.
+
+When using Scala in Stratosphere an important concept to grasp is that of the
+`DataSet`. `DataSet` is an abstract concept that represents actual data sets at
+runtime and which has operations that transform data to create a new transformed
+data set. In this example the `TextFile("/some/input")` call creates a
+`DataSet[String]` that represents the lines of text from the input. The
+`flatMap` operation that looks like a regular Scala flatMap is in fact an
+operation on `DataSet` that passes (at runtime) the data items through the
+provided anonymous function to transform them. The result of the `flatMap`
+operation is a new `DataSet` that represents the transformed data. On this other
+operations be performed. Another such operation are `groupBy` and `reduce`, but
+we will go into details of those later in this guide.
+
+The `write` operation of `DataSet` is used to create a data sink. You provide it
+with a path where the data is to be written to and an output format. This is
+enough for now but we will discuss data formats (for sources and sinks) later.
+
+To execute a data flow graph one or several sinks have to wrapped in a `Plan`
+which can then be executed on a cluster using `RemoteExecutor`. Here, the
+`LocalExecutor` is used to run the flow on the local computer. This is useful
+for debugging your job before running it on an actual cluster.
+
+<section id="intro-example">
+Project Setup
+-------------
+
+We will only cover maven here but the concepts should work equivalently with
+other build systems such as Gradle or sbt. When wanting to develop a Scala job
+all that is needed as dependency is is `stratosphere-scala` (and `stratosphere-clients`, if
+you want to execute your jobs). So all that needs to be done is to add the
+following lines to your POM.
+
+
+```xml
+<dependencies>
+  <dependency>
+    <groupId>eu.stratosphere</groupId>
+    <artifactId>stratosphere-scala</artifactId>
+    <version>{{site.current_stable}}</version>
+  </dependency>
+  <dependency>
+    <groupId>eu.stratosphere</groupId>
+    <artifactId>stratosphere-clients</artifactId>
+    <version>{{site.current_stable}}</version>
+  </dependency>
+</dependencies>
+```
+
+To quickly get started you can use the Stratosphere Scala quickstart available
+[here]({{site.baseurl}}/quickstart/scala.html). This will give you a
+completeMaven project with some working example code that you can use to explore
+the system or as basis for your own projects.
+
+These imports are normally enough for any project:
+
+```scala
+import eu.stratosphere.api.scala._
+import eu.stratosphere.api.scala.operators._
+
+import eu.stratosphere.client.LocalExecutor
+import eu.stratosphere.client.RemoteExecutor
+```
+
+The first two imports contain things like `DataSet`, `Plan`, data sources, data
+sinks, and the operations. The last two imports are required if you want to run
+a data flow on your local machine, respectively cluster.
+
+<section id="dataset">
+The DataSet Abstraction
+-----------------------
+
+As already alluded to in the introductory example you write Scala jobs by using
+operations on a `DataSet` to create new transformed `DataSet`. This concept is
+the core of the Stratosphere Scala API so it merits some more explanation. A
+`DataSet` can look and behave like a regular Scala collection in your code but
+it does not contain any actual data but only represents data. For example: when
+you use `TextFile()` you get back a `DataSource[String]` that represents each
+line of text in the input as a `String`. No data is actually loaded or available
+at this point. The set is only used to apply further operations which themselves
+are not executed until the data flow is executed. An operation on `DataSet`
+creates a new `DataSet` that represents the transformation and has a pointer to
+the `DataSet` that represents the data to be transformed. In this way a tree of
+data sets is created that contains both the specification of the flow of data as
+well as all the transformations. This graph can be wrapped in a `Plan` and
+executed.
+
+Working with the system is like working with lazy collections, where execution
+is postponed until the user submits the job.
+
+`DataSet` has a generic parameter, this is the type of each data item or record
+that would be processed by further transformations. This is similar to how
+`List[A]` in Scala would behave. For example in:
+
+```scala
+val input: DataSet[(String, Int)] = ...
+val mapped = input map { a => (a._1, a._2 + 1)}
+```
+
+The anonymous function would retrieve in `a` tuples of type `(String, Int)`.
+
+<section id="datatypes">
+Data Types
+----------
+
+There are some restrictions regarding the data types that can be used in Scala
+jobs (basically the generic parameter of `DataSet`). The usable types are
+the primitive Scala types, case classes (which includes tuples), and custom
+data types.
+
+Custom data types must implement the interface
+[Value](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/types/Value.java).
+For custom data types that should also be used as a grouping key or join key
+the [Key](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/types/Key.java)
+interface must be implemented.
+
+
+
+<section id="data-sources">
+Creating Data Sources
+---------------------
+
+To get an initial `DataSet` on which to perform operations to build a data flow
+graph the following construct is used:
+
+```scala
+val input = DataSource("<file-path>", <input-format>)
+```
+
+The value `input` is now a `DataSet` with the generic type depending on the
+input format.
+
+The file path can be on of either `file:///some/file` to acces files on the
+local machine or `hdfs://some/path` to read files from HDFS. The input
+format can be one of our builtin formats or a custom input format. The builtin
+formats are:
+
+* [TextInputFormat](#text-input-format)
+* [CsvInputFormat](#csv-input-format)
+* [DelimitedInputFormat](#delimited-input-format)
+* [BinaryInputFormat](#binary-input-format)
+* [BinarySerializedInputFormat](#binary-serialized-input-format)
+* [FixedLengthInputFormat](#fixed-length-input-format)
+
+We will now have a look at each of them and show how they are employed and in
+which situations.
+
+<section id="text-input-format">
+#### TextInputFormat
+
+This input format simply reads a text file line wise and creates a `String`
+for each line. It is used as:
+
+```scala
+TextInputFormat()
+```
+
+As you have already seen in the Word Count Example there is a shortcut for this.
+Instead of using a `DataSource` with `TextInputFormat` you can simply write:
+
+```scala
+val input = TextFile("<file-path>")
+```
+
+The `input` would then be a `DataSet[String]`.
+
+<section id="csv-input-format">
+#### CsvInputFormat
+
+This input format is mainly used to read Csv-Files, as the name suggests. Input
+files must be text files. You can specify the `String` that should be used
+as the separator between individual records (this would often be newline) and
+also the separator between fields of a record (this would often be a comma).
+The `CsvInputFormat` will automatically read the records and create
+Scala tuples or custom case class objects for you. The format can be used
+in one of the following ways:
+
+```scala
+CsvInputFormat[Out]()
+CsvInputFormat[Out](recordDelim: String)
+CsvInputFormat[Out](recordDelim: String, fieldDelim: Char)
+
+CsvInputFormat[Out](fieldIndices: Seq[Int])
+CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String)
+CsvInputFormat[Out](fieldIndices: Seq[Int], recordDelim: String, fieldDelim: Char)
+```
+
+The default record delimiter is a newline, the default field delimiter is a
+comma. The type parameter `Out` must be a case class type, which also includes
+tuple types since they are internally case classes.
+
+Normally, all the fields of a record are read. If you want to explicitly
+specify which fields of the record should be read you can use one of the
+tree variants with a `fieldIndices` parameter. Here you give a list
+of the fields that should be read. Field indices start from zero.
+
+An example usage could look as follows:
+
+```scala
+val input = DataSource("file:///some/file", CsvInputFormat[(Int, Int, String)](Seq(1, 17, 42), "\n", ','))
+```
+
+Here only the specified fields would be read and 3-tuples created for you.
+The type of input would be `DataSet[(Int, Int, String)]`.
+
+<section id="delimited-input-format">
+#### DelimitedInputFormat
+
+This input format is meant for textual records that are separated by
+some delimiter. The delimiter could be a newline, for example. It is used like
+this:
+
+```scala
+DelimitedInputFormat[Out](parseFunction: String => Out, delim: String = "\n")
+```
+
+The input files will be split on the supplied delimiter (or the default newline)
+and the supplied parse function must parse the textual representation in the
+`String` and return an object. The type of this object will then also be the
+type of the `DataSet` created by the `DataSource` operation.
+
+Just as with `BinaryInputFormat` the function can be an anonymous function, so
+you could have:
+
+```scala
+val input = DataSource("file:///some/file", BinaryInputFormat( { line =>
+  line match {
+    case EdgeInputPattern(from, to) => Path(from.toInt, to.toInt, 1)
+  }
+}))
+```
+
+In this example EdgeInputPattern is some regular expression used for parsing
+a line of text and `Path` is a custom case class that is used to represent
+the data. The type of input would in this case be `DataSet[Path]`.
+
+<section id="binary-input-format">
+#### BinaryInputFormat
+
+This input format is best used when you have a custom binary format that
+you store the data in. It is created using one of the following:
+
+```scala
+BinaryInputFormat[Out](readFunction: DataInput => Out)
+BinaryInputFormat[Out](readFunction: DataInput => Out, blocksize: Long)
+```
+
+So you have to provide a function that gets a
+[java.io.DataInput](http://docs.oracle.com/javase/7/docs/api/java/io/DataInput.html)
+and returns the object that
+contains the data. The type of this object will then also be the type of the
+`DataSet` created by the `DataSource` operation.
+
+The provided function can also be an anonymous function, so you could
+have something like this:
+
+```scala
+val input = DataSource("file:///some/file", BinaryInputFormat( { input =>
+  val one = input.readInt
+  val two = input.readDouble
+  (one, two)  
+}))
+```
+
+Here `input` would be of type `DataSet[(Int, Double)]`.
+
+<section id="binary-serialized-input-format">
+#### BinarySerializedInputFormat
+
+This input format is only meant to be used in conjunction with
+`BinarySerializedOutputFormat`. You can use these to write elements to files using a
+Stratosphere-internal format that can efficiently be read again. You should only
+use this when output is only meant to be consumed by other Stratosphere jobs.
+The format can be used on one of two ways:
+
+```scala
+BinarySerializedInputFormat[Out]()
+BinarySerializedInputFormat[Out](blocksize: Long)
+```
+
+So if input files contain elements of type `(String, Int)` (a tuple type) you
+could use:
+
+```scala
+val input = DataSource("file:///some/file", BinarySerializedInputFormat[(String, Int)]())
+```
+
+<section id="fixed-length-input-format">
+#### FixedLengthInputFormat
+
+This input format is for cases where you want to read binary blocks
+of a fixed size. The size of a block must be specified and you must
+provide code that reads elements from a byte array.
+
+The format is used like this:
+
+```scala
+FixedLengthInputFormat[Out](readFunction: (Array[Byte], Int) => Out, recordLength: Int)
+```
+
+The specified function gets an array and a position at which it must start
+reading the array and returns the element read from the binary data.
+
+
+<section id="operations">
+Operations on DataSet
+---------------------
+
+As explained in [Programming Model](pmodel.html#operators),
+a Stratosphere job is a graph of operators that process data coming from
+sources that is finally written to sinks. When you use the Scala front end
+these operators as well as the graph is created behind the scenes. For example,
+when you write code like this:
+
+```scala
+val input = TextFile("file:///some/file")
+
+val words = input.map { x => (x, 1) }
+
+val output = counts.write(words, CsvOutputFormat()))
+
+val plan = new ScalaPlan(Seq(output))
+```
+
+What you get is a graph that has a data source, a map operator (that contains
+the code written inside the anonymous function block), and a data sink. You 
+do not have to know about this to be able to use the Scala front end but
+it helps to remember, that when you are using Scala you are building
+a data flow graph that processes data only when executed.
+
+There are operations on `DataSet` that correspond to all the types of operators
+that the Stratosphere system supports. We will shortly go trough all of them with
+some examples.
+
+<section id="operator-templates">
+#### Basic Operator Templates
+
+Most of the operations have three similar versions and we will
+explain them here for all of the operators together. The three versions are `map`,
+`flatMap`, and `filter`. All of them accept an anonymous function that
+defines what the operation does but the semantics are different.
+
+The `map` version is a simple one to one mapping. Take a look at the following
+code:
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.map { x => (x._1, x._2 + 3) }
+```
+
+This defines a map operator that operates on tuples of String and Int and just
+adds three to the Int (the second fields of the tuple). So, if the input set had
+the tuples (a, 1), (b, 2), and (c, 3) the result after the operator would be
+(a, 4), (b, 5), and (c, 6).
+
+The `flatMap` version works a bit differently,
+here you return something iterable from the anonymous function. The iterable
+could be a list or an array. The elements in this iterable are unnested.
+So for every element in the input data you get a list of elements. The
+concatenation of those is the result of the operator. If you had
+the following code:
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.flatMap { x => List( (x._1, x._2), (x._1, x._2 + 1) ) }
+```
+
+and as input the tuples (a, 1) and (b, 1) you would get (a, 1), (a, 2), (b, 1),
+and (b, 2) as result. It is one flat list, and not the individual lists returned
+from the anonymous function.
+
+The third template is `filter`. Here you give an anonymous function that
+returns a Boolean. The elements for which this Boolean is true are part of the
+result of the operation, the others are culled. An example for a filter is this
+code:
+
+
+```scala
+val input: DataSet[(String, Int)]
+
+val mapped = input.filter { x => x._2 >= 3 }
+```
+
+<section id="key-selectors">
+#### Field/Key Selectors
+
+For some operations (group, join, and cogroup) it is necessary to specify which
+parts of a data type are to be considered the key. This key is used for grouping
+elements together for reduce and for joining in case of a join or cogroup.
+In Scala the key is specified using a special anonymous function called
+a key selector. The key selector has as input an element of the type of
+the `DataSet` and must return a single value or a tuple of values that should
+be considered the key. This will become clear with some examples: (Note that
+we use the reduce operation here as an example, we will have a look at
+that further down):
+
+```scala
+val input: DataSet[(String, Int)]
+val reduced = input groupBy { x => (x._1) } reduce { ... }
+val reduced2 = input groupBy { case (w, c) => w } reduce { ... }
+
+case class Test(a: String, b: Int, c: Int)
+val input2: DataSet[Test]
+val reduced3 = input2 groupBy { x => (x.a, x.b) } reduce { ... }
+val reduced4 = input2 groupBy { case Test(x,y,z) => (x,y) } reduce { ... }
+```
+
+The anonymous function block passed to `groupBy` is the key selector. The first
+two examples both specify the `String` field of the tuple as key. In the second
+set of examples we see a custom case class and here we select the first two
+fields as a compound key.
+
+It is worth noting that the key selector function is not actually executed 
+at runtime but it is parsed at job creation time where the key information is
+extracted and stored for efficient computation at runtime.
+
+#### Map Operation
+
+Map is an operation that gets one element at a time and can output one or
+several elements. The operations that result in a `MapOperator` in the graph are exactly
+those mentioned in the previous section. For completeness' sake we will mention
+their signatures here (in this and the following such lists `In` is the
+type of the input data set, `DataSet[In]`):
+
+```scala
+def map[Out](fun: In => Out): DataSet[Out]
+def flatMap[Out](fun: In => Iterator[Out]): DataSet[Out]
+def filter(fun: In => Boolean): DataSet[Out]
+```
+
+#### Reduce Operation
+
+As explained [here](pmodel.html#operators) Reduce is an operation that looks
+at groups of elements at a time and can, for one group, output one or several
+elements. To specify how elements should be grouped you need to give
+a key selection function, as explained [above](#key-selectors).
+
+The basic template of the reduce operation is:
+
+```scala
+input groupBy { <key selector> } reduce { <reduce function> }
+```
+
+The signature of the reduce function depends on the variety of reduce operation
+selected. There are right now three different versions:
+
+```scala
+def reduce(fun: (In, In) => In): DataSet[In]
+
+def reduceGroup[Out](fun: Iterator[In] => Out): DataSet[Out]
+def combinableReduceGroup(fun: Iterator[In] => In): DataSet[In]
+```
+
+The `reduce` variant is like a `reduceLeft` on a Scala collection with
+the limitation that the output data type must be the same as the input data
+type. You specify how to elements of the selection should be combined,
+this is then used to reduce the elements in one group (of the same key)
+down to one element. This can be used to implement aggregation operators,
+for example:
+
+```scala
+val words: DataSet[(String, Int)]
+val counts = words.groupBy { case (word, count) => word}
+  .reduce { (w1, w1) => (w1._1, w1._2 + w2._2) }
+```
+
+This would add up the Int fields of those tuples that have the same String
+in the first fields. As is for example required in Word Count.
+
+The `reduceGroup` variant can be used when more control is required. Here
+your reduce function gets an `Iterator` that can be used to iterate over
+all the elements in a group. With this type or reduce operation the
+output data type can be different from the input data type. An example
+of this kind of operation is this:
+
+```scala
+val words: DataSet[(String, Int)]
+val minCounts = words.groupBy { case (word, count) => word}
+  .reduceGroup { words => words.minBy { _._2 } }
+```
+
+Here we use the minBy function of Scala collections to determine the
+element with the minimum count in a group.
+
+The `combinableGroupReduce` works like the `groupReduce` with the difference
+that the reduce operation is combinable. This is an optimization one can use,
+please have a look at [Programming Model](pmodel.html "Programming Model") for
+the details.
+
+#### Join Operation
+
+The join operation is similar to a database equi-join. It is a two input
+iteration where you have to specify a key selector for each of the inputs
+and then the anonymous function is called for every pair of matching
+elements from the two input sides.
+
+The basic template is:
+
+```scala
+input1 join input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <join function> }
+```
+
+or, because lines will get to long fast:
+```scala
+input1.join(input2)
+  .where { <key selector 1> }
+  .isEqualTo { <key selector 2>}
+  .map { <join function> }
+```
+
+(Scala can sometimes be quite finicky about where you can omit dots and
+parentheses, so it's best to use dots in multi-line code like this.)
+
+As mentioned in [here](#operator-templates) there are three versions of
+this operator, so you can use one of these in the last position:
+
+```scala
+def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
+def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
+def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
+```
+
+One example where this can be used is database-style joining with projection:
+
+```scala
+input1.join(input2)
+  .where { case (a, b, c) => (a, b) }
+  .isEqualTo { case (a, b, c, d) => (c, d) }
+  .map { (left, right) => (left._3, right._1) }
+```
+
+Here the join key for the left input is a compound of the first two tuple fields
+while the key for the second input is a compound of the last two fields. We then
+pick one field each from both sides as the result of the operation.
+
+#### CoGroup Operation
+
+The cogroup operation is a cross between join and reduce. It has two inputs
+and you have to specify a key selector for each of them. This is where the
+similarities with join stop. Instead of having one invocation of your user
+code per pair of matching elements all elements from the left and from the right
+are grouped together for one single invocation. In your function you get
+an `Iterator` for the elements from the left input and another `Iterator`
+for the elements from the right input.
+
+The basic template is:
+
+```scala
+input1 cogroup input2 where { <key selector 1> } isEqualTo { <key selector 2>} map { <cogroup function> }
+```
+
+or, because lines will get to long fast:
+```scala
+input1.cogroup(input2)
+  .where { <key selector 1> }
+  .isEqualTo { <key selector 2>}
+  .map { <cogroup function> }
+```
+
+There are to variants you can use, with the semantics explained
+[here](#operator-templates).
+
+```scala
+def map[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Out): DataSet[Out]
+def flatMap[Out](fun: (Iterator[LeftIn], Iterator[RightIn]) => Iterator[Out]): DataSet[Out]
+```
+
+#### Cross Operation
+
+The cross operation is used to form the Cartesian product of the elements
+from two inputs. The basic template is:
+
+```scala
+input1 cross input2 map { <cogroup function> }
+```
+
+Again there are three variants, with the semantics explained
+[here](#operator-templates).
+
+```scala
+def map[Out](fun: (LeftIn, RightIn) => Out): DataSet[Out]
+def flatMap[Out](fun: (LeftIn, RightIn) => Iterator[Out]): DataSet[Out]
+def filter(fun: (LeftIn, RightIn) => Boolean): DataSet[(LeftIn, RightIn)]
+```
+
+#### Union
+
+When you want to have the combination of several data sets as the input of
+an operation you can use a union to combine them. It is used like this
+
+```scala
+val input1: DataSet[String]
+val input2: DataSet[String]
+val unioned = input1.union(input2)
+```
+
+The signature of union is:
+
+```scala
+def union(secondInput: DataSet[A])
+```
+
+Where `A` is the generic type of the `DataSet` on which you execute the `union`.
+
+<section id="iterations">
+Iterations
+----------
+
+Iterations allow you to implement *loops* in Stratosphere programs.
+[This page](iterations.html) gives a
+general introduction to iterations. This section here provides quick examples
+of how to use the concepts using the Scala API.
+The iteration operators encapsulate a part of the program and execute it
+repeatedly, feeding back the result of one iteration (the partial solution) into
+the next iteration. Stratosphere has two different types of iterations,
+*Bulk Iteration* and *Delta Iteration*.
+
+For both types of iterations you provide the iteration body as a function
+that has data sets as input and returns a new data set. The difference is
+that bulk iterations map from one data set two one new data set while
+delta iterations map two data sets to two new data sets.
+
+#### Bulk Iteration
+
+The signature of the bulk iterate method is this:
+
+```scala
+def iterate(n: Int, stepFunction: DataSet[A] => DataSet[A])
+```
+
+where `A` is the type of the `DataSet` on which `iterate` is called. The number
+of steps is given in `n`. This is how you use it in practice:
+
+```scala
+val dataPoints = DataSource(dataPointInput, DelimitedInputFormat(parseInput))
+val clusterPoints = DataSource(clusterInput, DelimitedInputFormat(parseInput))
+
+def kMeansStep(centers: DataSet[(Int, Point)]) = {
+
+  val distances = dataPoints cross centers map computeDistance
+  val nearestCenters = distances.groupBy { case (pid, _) => pid }
+    .reduceGroup { ds => ds.minBy(_._2.distance) } map asPointSum.tupled
+  val newCenters = nearestCenters.groupBy { case (cid, _) => cid }
+    .reduceGroup sumPointSums map { case (cid, pSum) => cid -> pSum.toPoint() }
+
+  newCenters
+}
+
+val finalCenters = clusterPoints.iterate(numIterations, kMeansStep)
+
+val output = finalCenters.write(clusterOutput, DelimitedOutputFormat(formatOutput.tupled))
+```
+
+Not that we use some functions here which we don't show. If you want, you
+can check out the complete code in our KMeans example.
+
+#### Delta Iteration
+
+The signature of the delta iterate method is this:
+
+```scala
+def iterateWithDelta(workset: DataSet[W], solutionSetKey: A => K, stepFunction: (DataSet[A], DataSet[W]) => (DataSet[A], DataSet[W]), maxIterations: Int)
+```
+
+where `A` is the type of the `DataSet` on which `iterateWithDelta` is called,
+`W` is the type of the `DataSet` that represents the workset and `K` is the
+key type. The maximum number of iterations must always be given.
+
+For information on how delta iterations in general work on our system, please
+refer to [iterations](iterations.html). A working example job is
+available here:
+[Scala Connected Components Example](examples_scala.html#connected_components) 
+
+
+<section id="data-sinks">
+Creating Data Sinks
+-------------------
+
+The creation of data sinks is analog to the creation of data sources. `DataSet`
+has a `write` method that is used to create a sink that writes the output
+of the operation to a file in the local file system or HDFS. The general pattern
+is this:
+
+```scala
+val sink = out.write("<file-path>", <output-format>)
+```
+
+Where `out` is some `DataSet`. Just as for data sources, the file path can be
+on of either `file:///some/file` to acces files on the local machine or
+`hdfs://some/path` to read files from HDFS. The output format can be one of our
+builtin formats or a custom output format. The builtin formats are:
+
+* [DelimitedOutputFormat](#delimited-output-format)
+* [CsvOutputFormat](#csv-output-format)
+* [RawOutputFormat](#raw-output-format)
+* [BinaryOutputFormat](#binary-output-format)
+* [BinarySerializedOutputFormat](#binary-serialized-output-format)
+
+We will now have a look at each of them and show how they are employed and in
+which situations.
+
+<section id="delimited-output-format">
+#### DelimitedOutputFormat
+
+This output format is meant for writing textual records that are separated by
+some delimiter. The delimiter could be a newline, for example. It is used like
+this:
+
+```scala
+DelimitedOutputFormat[In](formatFunction: In => String, delim: String = "\n")
+```
+
+For every element in the `DataSet` the formatting function is called and
+the result of that is appended to the output file. In between the elements
+the `delim` string is inserted.
+
+An example would be:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", DelimitedOutputFormat( { elem =>
+  "%s|%d".format(elem._1, elem._2)
+}))
+```
+
+Here we use Scala String formatting to write the two fields of the tuple
+separated by a pipe character. The default newline delimiter will be inserted
+between the elements in the output files.
+
+<section id="csv-output-format">
+#### CsvOutputFormat
+
+This output format can be used to automatically write fields of tuple
+elements or case classes to CSV files. You can specify what separator should
+be used between fields of an element and also the separator between elements.
+
+```scala
+CsvOutputFormat[In]()
+CsvOutputFormat[In](recordDelim: String)
+CsvOutputFormat[In](recordDelim: String, fieldDelim: Char)
+```
+
+The default record delimiter is a newline, the default field delimiter is a
+comma. 
+
+An example usage could look as follows:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", CsvOutputFormat())
+```
+
+Notice how we don't need to specify the generic type here, it is inferred.
+
+<section id="raw-output-format">
+#### RawOutputFormat
+
+This input format can be used when you want to have complete control over
+what gets written. You get an
+[OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
+and can write the elements of the `DataSet` exactly as you see fit.
+
+A `RawOutputFormat` is created like this:
+
+```scala
+RawOutputFormat[In](writeFunction: (In, OutputStream) => Unit)
+```
+
+The function you pass in gets one element from the `DataSet` and must
+write it to the given `OutputStream`. An example would be the following:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", RawOutputFormat( { (elem, output) =>
+  /* write elem._1 and elem._2 to output */ 
+}))
+```
+
+<section id="binary-output-format">
+#### BinaryOutputFormat
+
+This format is very similar to the RawOutputFormat. The difference is that
+instead of an [OutputStream](http://docs.oracle.com/javase/7/docs/api/java/io/OutputStream.html)
+you get a [DataOutput](http://docs.oracle.com/javase/7/docs/api/java/io/DataOutput.html)
+to which you can write binary data. You can also specify the block size for
+the binary output file. When you don't specify a block size some default
+is used.
+
+A `BinaryOutputFormat` is created like this:
+
+```scala
+BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit)
+BinaryOutputFormat[In](writeFunction: (In, DataOutput) => Unit, blockSize: Long)
+```
+
+<section id="binary-serialized-output-format">
+#### BinarySerializedOutputFormat
+
+This output format is only meant to be used in conjunction with
+`BinarySerializedInputFormat`. You can use these to write elements to files using a
+Stratosphere-internal format that can efficiently be read again. You should only
+use this when output is only meant to be consumed by other Stratosphere jobs.
+The output format can be used on one of two ways:
+
+```scala
+BinarySerializedOutputFormat[In]()
+BinarySerializedOutputFormat[In](blocksize: Long)
+```
+
+So to write elements of some `DataSet[A]` to a binary file you could use:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", BinarySerializedInputFormat())
+```
+
+As you can see the type of the elements need not be specified, it is inferred
+by Scala.
+
+<section id="execution">
+Executing Jobs
+--------------
+
+To execute a data flow graph the sinks need to be wrapped in a
+[ScalaPlan](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-scala/src/main/scala/eu/stratosphere/api/scala/ScalaPlan.scala)
+object like this:
+
+```scala
+val out: DataSet[(String, Int)]
+val sink = out.write("file:///some/file", CsvOutputFormat())
+
+val plan = new ScalaPlan(Seq(sink))
+```
+
+You can put several sinks into the `Seq` that is passed to the constructor.
+
+There are two ways one can execute a data flow plan: local execution and
+remote/cluster execution. When using local execution the plan is executed on
+the local computer. This is handy while developing jobs because you can
+easily debug your code and iterate quickly. When a job is ready to be
+used on bigger data sets it can be executed on a cluster. We will
+now give an example for each of the two execution modes.
+
+First up is local execution:
+
+```scala
+import eu.stratosphere.client.LocalExecutor
+
+...
+
+val plan: ScalaPlan = ...
+LocalExecutor.execute(plan)
+```
+
+This is all there is to it.
+
+Remote (or cluster) execution is a bit more complicated because you have
+to package your code in a jar file so that it can be distributed on the cluster.
+Have a look at the [scala quickstart](/quickstart/scala.html) to see how you
+can set up a maven project that does the packaging. Remote execution is done
+using the [RemoteExecutor](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-clients/src/main/java/eu/stratosphere/client/RemoteExecutor.java), like this:
+
+```scala
+import eu.stratosphere.client.RemoteExecutor
+
+...
+
+val plan: ScalaPlan = ...
+val ex = new RemoteExecutor("<job manager ip address>", <job manager port>, "your.jar");
+ex.executePlan(plan);
+```
+
+The IP address and the port of the Stratosphere job manager depend on your
+setup. Have a look at [cluster quickstart](/quickstart/setup.html) for a quick
+guide about how to set up a cluster. The default cluster port is 6123, so
+if you run a job manger on your local computer you can give this and "localhost"
+as the first to parameters to the `RemoteExecutor` constructor.
+
+<section id="rich-functions">
+Rich Functions
+--------------
+
+Sometimes having a single function that is passed to an operation is not enough.
+Using Rich Functions it is possible to have state inside your operations and
+have code executed before the first element is processed and after the last
+element is processed. For example, instead of a simple function as in this
+example:
+
+```scala
+val mapped = input map { x => x + 1 }
+```
+
+you can have a rich function like this:
+
+```scala
+val mapped = input map( new MapFunction[(String, Int), (String, Int)] {
+  val someState: SomeType = ...
+  override def open(config: Configuration) = {
+    // one-time initialization code
+  }
+
+  override def close() = {
+    // one-time clean-up code
+  }
+
+  override def apply(in: (String, Int)) = {
+    // do complex stuff
+    val result = ...
+    result
+  }
+})
+```
+
+You could also create a custom class that derives from `MapFunction`
+instead of the anonymous class we used here.
+
+There are rich functions for all the various operator types. The basic
+template is the some, though. The common interface that they implement 
+is [Function](https://github.com/stratosphere/stratosphere/blob/release-{{site.current_stable}}/stratosphere-core/src/main/java/eu/stratosphere/api/common/functions/Function.java). The `open` and `close` methods can be overridden to run set-up
+and tear-down code. The other methods can be used in a rich function to
+work with the runtime context which gives information about the context
+of the operator. Your operation code must now reside in an `apply` method
+that has the same signature as the anonymous function you would normally
+supply.
+
+The rich functions reside in the package `eu.stratosphere.api.scala.functions`.
+This is a list of all the rich functions can can be used instead of
+simple functions in the respective operations:
+
+```scala
+abstract class MapFunction[In, Out] 
+abstract class FlatMapFunction[In, Out] 
+abstract class FilterFunction[In, Out] 
+
+abstract class ReduceFunction[In]
+abstract class GroupReduceFunction[In, Out]
+abstract class CombinableGroupReduceFunction[In, Out]
+
+abstract class JoinFunction[LeftIn, RightIn, Out]
+abstract class FlatJoinFunction[LeftIn, RightIn, Out]
+
+abstract class CoGroupFunction[LeftIn, RightIn, Out]
+abstract class FlatCoGroupFunction[LeftIn, RightIn, Out]
+
+abstract class CrossFunction[LeftIn, RightIn, Out]
+abstract class FlatCrossFunction[LeftIn, RightIn, Out]
+```
+
+Note that for all the rich stubs, you need to specify the generic type of
+the input (or inputs) and the output type.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/scala_api_quickstart.md b/docs/scala_api_quickstart.md
new file mode 100644
index 0000000..e15eed0
--- /dev/null
+++ b/docs/scala_api_quickstart.md
@@ -0,0 +1,71 @@
+---
+title: "Quick Start: Scala API"
+---
+
+<p class="lead">Start working on your Stratosphere Scala program in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">The only requirements are working <strong>Maven 3.0.4</strong> (or higher) and <strong>Java 6.x</strong> (or higher) installations.</p>
+</section>
+
+<section id="create_project">
+  <div class="page-header"><h2>Create Project</h2></div>
+  <p class="lead">Use one of the following commands to <strong>create a project</strong>:</p>
+
+  <ul class="nav nav-tabs" style="border-bottom: none;">
+      <li class="active"><a href="#quickstart-script" data-toggle="tab">Run the <strong>quickstart script</strong></a></li>
+      <li><a href="#maven-archetype" data-toggle="tab">Use <strong>Maven archetypes</strong></a></li>
+  </ul>
+  <div class="tab-content">
+      <div class="tab-pane active" id="quickstart-script">
+{% highlight bash %}
+$ curl https://raw.githubusercontent.com/stratosphere/stratosphere-quickstart/master/quickstart-scala.sh | bash
+{% endhighlight %}
+      </div>
+      <div class="tab-pane" id="maven-archetype">
+{% highlight bash %}
+$ mvn archetype:generate                             \
+    -DarchetypeGroupId=eu.stratosphere               \
+    -DarchetypeArtifactId=quickstart-scala           \
+    -DarchetypeVersion={{site.current_stable}}                  
+{% endhighlight %}
+      This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
+      </div>
+  </div>
+</section>
+
+<section id="inspect_project">
+  <div class="page-header"><h2>Inspect Project</h2></div>
+  <p class="lead">There will be a <strong>new directory in your working directory</strong>. If you've used the <em>curl</em> approach, the directory is called <code>quickstart</code>. Otherwise, it has the name of your artifactId.</p>
+  <p class="lead">The sample project is a <strong>Maven project</strong>, which contains a sample scala <em>Job</em> that implements Word Count. Please note that the <em>RunJobLocal</em> and <em>RunJobRemote</em> objects allow you to start Stratosphere in a development/testing mode.</p>
+  <p class="lead">We recommend to <strong>import this project into your IDE</strong>. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites:
+    <ul>
+      <li class="lead"><strong>Eclipse 4.x</strong>:
+        <ul>
+          <li><strong>Scala IDE</strong> <small>(http://download.scala-ide.org/sdk/e38/scala210/stable/site)</small></li>
+          <li><strong>m2eclipse-scala</strong> <small>(http://alchim31.free.fr/m2e-scala/update-site)</small></li>
+          <li><strong>Build Helper Maven Plugin</strong> <small>(https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.15.0/N/0.15.0.201206251206/)</small></li>
+        </ul>
+      </li>
+      <li class="lead"><strong>Eclipse 3.7</strong>:
+        <ul>
+          <li><strong>Scala IDE</strong> <small>(http://download.scala-ide.org/sdk/e37/scala210/stable/site)</small></li>
+          <li><strong>m2eclipse-scala</strong> <small>(http://alchim31.free.fr/m2e-scala/update-site)</small></li>
+          <li><strong>Build Helper Maven Plugin</strong> <small>(https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.14.0/N/0.14.0.201109282148/)</small></li>
+        </ul>
+      </li>
+    </ul>
+  </p>
+  <p class="lead">The IntelliJ IDE also supports Maven and offers a plugin for Scala development.</p>
+</section>
+
+<section id="build_project">
+  <div class="page-header"><h2>Build Project</h2></div>
+  <p class="lead">If you want to <strong>build your project</strong>, go to your project directory and issue the <code>mvn clean package</code> command. You will <strong>find a jar</strong> that runs on every Stratosphere cluster in <code>target/stratosphere-project-0.1-SNAPSHOT.jar</code>.</p>
+</section>
+
+<section id="next_steps">
+  <div class="page-header"><h2>Next Steps</h2></div>
+  <p class="lead"><strong>Write your application!</strong> If you have any trouble, ask on our <a href="https://github.com/stratosphere/stratosphere/issues">GitHub page</a> (open an issue) or on our <a href="https://groups.google.com/forum/#!forum/stratosphere-dev">Mailing list</a>. We are happy to provide help.</p>
+</p>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/40b94f73/docs/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/setup_quickstart.md b/docs/setup_quickstart.md
new file mode 100644
index 0000000..debe21c
--- /dev/null
+++ b/docs/setup_quickstart.md
@@ -0,0 +1,132 @@
+---
+title: "Quickstart: Setup"
+---
+
+<p class="lead">Get Stratosphere up and running in a few simple steps.</p>
+
+<section id="requirements">
+  <div class="page-header"><h2>Requirements</h2></div>
+  <p class="lead">Stratosphere runs on all <em>UNIX-like</em> environments: <strong>Linux</strong>, <strong>Mac OS X</strong>, <strong>Cygwin</strong>. The only requirement is to have a working <strong>Java 6.x</strong> (or higher) installation.</p>
+</section>
+
+<section id="download">
+  <div class="page-header"><h2>Download</h2></div>
+  <p class="lead">Download the ready to run binary package. Choose the Stratosphere distribution that <strong>matches your Hadoop version</strong>. If you are unsure which version to choose or you just want to run locally, pick the package for Hadoop 1.2.</p>
+  <p>
+  	<ul class="nav nav-tabs">
+  		<li class="active"><a href="#bin-hadoop1" data-toggle="tab">Hadoop 1.2</a></li>
+      <li><a href="#bin-hadoop2" data-toggle="tab">Hadoop 2 (YARN)</a></li>
+		</ul>
+		<div class="tab-content text-center">
+			<div class="tab-pane active" id="bin-hadoop1">
+				<a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-1',this.href]);" href="{{site.current_stable_dl}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 1.2</a>
+	    </div>
+			<div class="tab-pane" id="bin-hadoop2">
+	      <a class="btn btn-info btn-lg" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-setup-2',this.href]);" href="{{site.current_stable_dl_yarn}}"><i class="icon-download"> </i> Download Stratosphere for Hadoop 2 (YARN)</a>
+	    </div>
+	  </div>
+	</p>
+</section>
+
+<section id="start">
+  <div class="page-header"><h2>Start</h2></div> 
+  <p class="lead">You are almost done.</p>
+  <ol>
+  	<li class="lead"><strong>Go to the download directory</strong>,</li>
+  	<li class="lead"><strong>Unpack the downloaded archive</strong>, and</li>
+  	<li class="lead"><strong>Start Stratosphere</strong>.</li>
+  </ol>
+
+{% highlight bash %}
+$ cd ~/Downloads              # Go to download directory
+$ tar xzf stratosphere-*.tgz  # Unpack the downloaded archive
+$ cd stratosphere
+$ bin/start-local.sh          # Start Stratosphere
+{% endhighlight %}
+
+  <p class="lead">Check the <strong>JobManager's web frontend</strong> at <a href="http://localhost:8081">http://localhost:8081</a> and make sure everything is up and running.</p>
+</section>
+
+<section id="example">
+  <div class="page-header"><h2>Run Example</h2></div>
+  <p class="lead">Run the <strong>Word Count example</strong> to see Stratosphere at work.</p>
+
+  <ol>
+  	<li class="lead"><strong>Download test data:</strong>
+{% highlight bash %}
+$ wget -O hamlet.txt http://www.gutenberg.org/cache/epub/1787/pg1787.txt
+{% endhighlight %}
+		  You now have a text file called <em>hamlet.txt</em> in your working directory.
+		</li>
+  	<li class="lead"><strong>Start the example program</strong>:
+{% highlight bash %}
+$ bin/stratosphere run \
+    --jarfile ./examples/stratosphere-java-examples-{{site.current_stable}}-WordCount.jar \
+    --arguments file://`pwd`/hamlet.txt file://`pwd`/wordcount-result.txt
+{% endhighlight %}
+      You will find a file called <strong>wordcount-result.txt</strong> in your current directory.
+  	</li>
+  </ol>
+
+</section>
+
+<section id="cluster">
+  <div class="page-header"><h2>Cluster Setup</h2></div>
+  <p class="lead"><strong>Running Stratosphere on a cluster</strong> is as easy as running it locally. Having <strong>passwordless SSH</strong> and <strong>the same directory structure</strong> on all your cluster nodes lets you use our scripts to control everything.</p>
+  <ol>
+  	<li class="lead">Copy the unpacked <strong>stratosphere</strong> directory from the downloaded archive to the same file system path on each node of your setup.</li>
+  	<li class="lead">Choose a <strong>master node</strong> (JobManager) and set the <code>jobmanager.rpc.address</code> key in <code>conf/stratosphere-conf.yaml</code> to its IP or hostname. Make sure that all nodes in your cluster have the same <code>jobmanager.rpc.address</code> configured.</li>
+  	<li class="lead">Add the IPs or hostnames (one per line) of all <strong>worker nodes</strong> (TaskManager) to the slaves files in <code>conf/slaves</code>.</li>
+  </ol>
+  <p class="lead">You can now <strong>start the cluster</strong> at your master node with <code>bin/start-cluster.sh</code>.</p>
+  <p class="lead">
+    The following <strong>example</strong> illustrates the setup with three nodes (with IP addresses from <em>10.0.0.1</em> to <em>10.0.0.3</em> and hostnames <em>master</em>, <em>worker1</em>, <em>worker2</em>) and shows the contents of the configuration files, which need to be accessible at the same path on all machines:
+  </p>
+  <div class="row">
+    <div class="col-md-6 text-center">
+      <img src="{{ site.baseurl }}/img/quickstart_cluster.png" style="width: 85%">
+    </div>
+    <div class="col-md-6">
+      <div class="row">
+        <p class="lead text-center">
+        /path/to/<strong>stratosphere/conf/<br>stratosphere-conf.yaml</strong>
+<pre>
+jobmanager.rpc.address: 10.0.0.1
+</pre>
+        </p>
+      </div>
+      <div class="row" style="margin-top: 1em;">
+        <p class="lead text-center">
+        /path/to/<strong>stratosphere/<br>conf/slaves</strong>
+<pre>
+10.0.0.2
+10.0.0.3
+</pre>
+        </p>
+      </div>
+    </div>
+  </div>
+</section>
+
+<section id="yarn">
+  <div class="page-header"><h2>Stratosphere on YARN</h2></div>
+  <p class="lead">You can easily deploy Stratosphere on your existing <strong>YARN cluster</strong>. 
+    <ol>
+    <li class="lead">Download the <strong>Stratosphere YARN package</strong> with the YARN client:
+      <div class="text-center" style="padding: 1em;">
+      <a style="padding-left:10px" onclick="_gaq.push(['_trackEvent','Action','download-quickstart-yarn',this.href]);" class="btn btn-info btn-lg" href="{{site.current_stable_uberjar}}"><i class="icon-download"> </i> Stratosphere {{ site.current_stable }} for YARN</a>
+      </div>
+    </li>
+    <li class="lead">Make sure your <strong>HADOOP_HOME</strong> (or <em>YARN_CONF_DIR</em> or <em>HADOOP_CONF_DIR</em>) <strong>environment variable</strong> is set to read your YARN and HDFS configuration.</li>
+    <li class="lead">Run the <strong>YARN client</strong> with:
+      <div class="text-center" style="padding:1em;">
+        <code>./bin/yarn-session.sh</code>
+      </div>
+      
+      You can run the client with options <code>-n 10 -tm 8192</code> to allocate 10 TaskManagers with 8GB of memory each.</li>
+  </ol>
+  </p>
+</section>
+
+<hr />
+<p class="lead">For <strong>more detailed instructions</strong>, check out the <a href="{{site.baseurl}}/docs/{{site.current_stable_documentation}}">Documentation</a>.</p>
\ No newline at end of file