You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2014/06/27 22:32:19 UTC

[1/5] git commit: Fix the managed memory planning after patch 9ce6293075d1a2326df8a2e99c032445a555b28b

Repository: incubator-flink
Updated Branches:
  refs/heads/master 9ce629307 -> 5b390d546


Fix the managed memory planning after patch 9ce6293075d1a2326df8a2e99c032445a555b28b


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

Branch: refs/heads/master
Commit: 0d0803f57ad05999047b0a5eeddb59bda7a48182
Parents: 9ce6293
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 27 18:00:32 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Jun 27 18:10:14 2014 +0200

----------------------------------------------------------------------
 .../client/minicluster/NepheleMiniCluster.java  |  9 +++--
 .../nephele/taskmanager/TaskManager.java        | 40 ++++++++++++--------
 2 files changed, 30 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0d0803f5/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
----------------------------------------------------------------------
diff --git a/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java b/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
index 86a6efa..76ee1f9 100644
--- a/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
+++ b/stratosphere-clients/src/main/java/eu/stratosphere/client/minicluster/NepheleMiniCluster.java
@@ -272,7 +272,7 @@ public class NepheleMiniCluster {
 		config.setBoolean(ConfigConstants.FILESYSTEM_DEFAULT_OVERWRITE_KEY, defaultOverwriteFiles);
 		config.setBoolean(ConfigConstants.FILESYSTEM_OUTPUT_ALWAYS_CREATE_DIRECTORY_KEY, defaultAlwaysCreateDirectory);
 
-		if(memorySize < 0){
+		if (memorySize < 0){
 			memorySize = HardwareDescriptionFactory.extractFromSystem().getSizeOfFreeMemory();
 
 			// at this time, we need to scale down the memory, because we cannot dedicate all free memory to the
@@ -282,9 +282,12 @@ public class NepheleMiniCluster {
 					GlobalConfiguration.getLong(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
 							ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
 
-			memorySize = (long) (0.8 * (memorySize - bufferMem));
+			memorySize = memorySize - bufferMem;
+			
+			// apply the fraction that makes sure memory is left to the heap for other data structures and UDFs.
+			memorySize = (long) (memorySize * ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
 
-			//convert from bytes to mega bytes
+			//convert from bytes to megabytes
 			memorySize >>>= 20;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0d0803f5/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 0c5e218..58da664 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -334,9 +334,8 @@ public class TaskManager implements TaskOperationProtocol {
 			throw new Exception("Failed to instantiate ChannelManager.", ioe);
 		}
 
+		// initialize the number of slots
 		{
-			HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
-			
 			int slots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, -1);
 			if (slots == -1) {
 				slots = 1;
@@ -347,23 +346,34 @@ public class TaskManager implements TaskOperationProtocol {
 				LOG.info("Creating " + slots + " task slot(s).");
 			}
 			this.numberOfSlots = slots;
+		}
+		
+		this.hardwareDescription = HardwareDescriptionFactory.extractFromSystem();
+		
+		// initialize the memory manager
+		{
+			// Check whether the memory size has been explicitly configured.
+			final long configuredMemorySize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1);
+			final long memorySize;
 			
-			// Check whether the memory size has been explicitly configured. if so that overrides the default mechanism
-			// of taking as much as is mentioned in the hardware description
-			long memorySize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1);
-
-			if (memorySize > 0) {
-				// manually configured memory size. override the value in the hardware config
-				resources = HardwareDescriptionFactory.construct(resources.getNumberOfCPUCores(),
-					resources.getSizeOfPhysicalMemory(), memorySize * 1024L * 1024L);
+			if (configuredMemorySize == -1) {
+				// no manually configured memory. take a relative fraction of the free heap space
+				float fraction = GlobalConfiguration.getFloat(ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+				memorySize = (long) (this.hardwareDescription.getSizeOfFreeMemory() * fraction);
+				LOG.info("Using " + fraction + " of the free heap space for managed memory.");
+			}
+			else if (configuredMemorySize <= 0) {
+				throw new Exception("Invalid value for Memory Manager memory size: " + configuredMemorySize);
+			}
+			else {
+				memorySize = configuredMemorySize << 20;
 			}
-			this.hardwareDescription = resources;
 
 			final int pageSize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
 					ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
 
 			// Initialize the memory manager
-			LOG.info("Initializing memory manager with " + (resources.getSizeOfFreeMemory() >>> 20) + " megabytes of memory. " +
+			LOG.info("Initializing memory manager with " + (memorySize >>> 20) + " megabytes of memory. " +
 					"Page size is " + pageSize + " bytes.");
 			
 			try {
@@ -371,11 +381,9 @@ public class TaskManager implements TaskOperationProtocol {
 				final boolean lazyAllocation = GlobalConfiguration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY,
 					ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_LAZY_ALLOCATION);
 				
-				this.memoryManager = new DefaultMemoryManager(resources.getSizeOfFreeMemory(), this.numberOfSlots,
-						pageSize);
+				this.memoryManager = new DefaultMemoryManager(memorySize, this.numberOfSlots, pageSize);
 			} catch (Throwable t) {
-				LOG.fatal("Unable to initialize memory manager with " + (resources.getSizeOfFreeMemory() >>> 20)
-					+ " megabytes of memory.", t);
+				LOG.fatal("Unable to initialize memory manager with " + (memorySize >>> 20) + " megabytes of memory.", t);
 				throw new Exception("Unable to initialize memory manager.", t);
 			}
 		}


[2/5] git commit: Fix checkstyle errors and various warnings.

Posted by se...@apache.org.
Fix checkstyle errors and various warnings.


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

Branch: refs/heads/master
Commit: 3c5feae7da1a52cb928bccf7bee43d4f98faf2c5
Parents: 0d0803f
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 27 20:37:09 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Jun 27 21:22:02 2014 +0200

----------------------------------------------------------------------
 .../src/main/java/eu/stratosphere/api/java/operators/Keys.java   | 1 +
 .../api/java/typeutils/runtime/PojoPairComparator.java           | 1 +
 .../eu/stratosphere/api/java/operator/CoGroupOperatorTest.java   | 2 --
 .../java/eu/stratosphere/api/java/operator/GroupingTest.java     | 2 --
 .../java/eu/stratosphere/api/java/operator/JoinOperatorTest.java | 3 ---
 .../api/java/type/extractor/PojoTypeInformationTest.java         | 4 +---
 .../api/java/typeutils/runtime/PojoSerializerTest.java           | 2 --
 .../nephele/instance/HardwareDescriptionFactory.java             | 2 --
 .../eu/stratosphere/test/javaApiOperators/DistinctITCase.java    | 1 -
 9 files changed, 3 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java
index 8e879d7..3477e00 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/Keys.java
@@ -187,6 +187,7 @@ public abstract class Keys<T> {
 
 		private final TypeInformation<?>[] types;
 
+		@SuppressWarnings("unused")
 		private PojoTypeInfo<?> type;
 
 		public ExpressionKeys(String[] expressions, TypeInformation<T> type) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/PojoPairComparator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/PojoPairComparator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/PojoPairComparator.java
index e8b4240..c862f13 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/PojoPairComparator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/typeutils/runtime/PojoPairComparator.java
@@ -28,6 +28,7 @@ public class PojoPairComparator<T1, T2> extends TypePairComparator<T1, T2> imple
 
 	private static final long serialVersionUID = 1L;
 
+	@SuppressWarnings("unused")
 	private final int[] keyPositions1, keyPositions2;
 	private transient Field[] keyFields1, keyFields2;
 	private final TypeComparator<Object>[] comparators1;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/CoGroupOperatorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/CoGroupOperatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/CoGroupOperatorTest.java
index b17e4b9..e4cc949 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/CoGroupOperatorTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/CoGroupOperatorTest.java
@@ -18,8 +18,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
-import eu.stratosphere.api.java.typeutils.TypeExtractor;
-import eu.stratosphere.types.TypeInformation;
 import junit.framework.Assert;
 
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/GroupingTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/GroupingTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/GroupingTest.java
index 6e7fb65..ad51361 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/GroupingTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/GroupingTest.java
@@ -18,8 +18,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
-import eu.stratosphere.api.java.typeutils.TypeExtractor;
-import eu.stratosphere.types.TypeInformation;
 import junit.framework.Assert;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/JoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/JoinOperatorTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/JoinOperatorTest.java
index f5d5dee..32747c1 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/JoinOperatorTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/operator/JoinOperatorTest.java
@@ -14,13 +14,10 @@
  **********************************************************************************************************************/
 package eu.stratosphere.api.java.operator;
 
-import java.beans.Expression;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
-import eu.stratosphere.api.java.typeutils.TypeExtractor;
-import eu.stratosphere.types.TypeInformation;
 import junit.framework.Assert;
 
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/PojoTypeInformationTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/PojoTypeInformationTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/PojoTypeInformationTest.java
index 065ae0c..772050a 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/PojoTypeInformationTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/type/extractor/PojoTypeInformationTest.java
@@ -13,17 +13,15 @@
 
 package eu.stratosphere.api.java.type.extractor;
 
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;
 
 import eu.stratosphere.api.java.typeutils.PojoTypeInfo;
 import eu.stratosphere.api.java.typeutils.TypeExtractor;
-import eu.stratosphere.types.Record;
-import eu.stratosphere.types.StringValue;
 import eu.stratosphere.types.TypeInformation;
 
+@SuppressWarnings("unused")
 public class PojoTypeInformationTest {
 
 	static class SimplePojo {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/PojoSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/PojoSerializerTest.java b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/PojoSerializerTest.java
index 8adb83b..c66e314 100644
--- a/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/PojoSerializerTest.java
+++ b/stratosphere-java/src/test/java/eu/stratosphere/api/java/typeutils/runtime/PojoSerializerTest.java
@@ -17,8 +17,6 @@ package eu.stratosphere.api.java.typeutils.runtime;
 import java.util.Random;
 
 import com.google.common.base.Objects;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
 
 import eu.stratosphere.api.common.typeutils.SerializerTestBase;
 import eu.stratosphere.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescriptionFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescriptionFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescriptionFactory.java
index 955c0c9..b633032 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescriptionFactory.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescriptionFactory.java
@@ -23,8 +23,6 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.GlobalConfiguration;
 import eu.stratosphere.util.OperatingSystem;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3c5feae7/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
index 11fcf97..28caed8 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/javaApiOperators/DistinctITCase.java
@@ -14,7 +14,6 @@
  **********************************************************************************************************************/
 package eu.stratosphere.test.javaApiOperators;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.LinkedList;


[4/5] Add first files for system internals documentation

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5b390d54/docs/internals/stack.svg
----------------------------------------------------------------------
diff --git a/docs/internals/stack.svg b/docs/internals/stack.svg
new file mode 100644
index 0000000..e45227c
--- /dev/null
+++ b/docs/internals/stack.svg
@@ -0,0 +1,587 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<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"
+   version="1.0"
+   width="221.08mm"
+   height="174.5mm"
+   id="svg3187"
+   inkscape:version="0.48.4 r9939"
+   sodipodi:docname="Picture2.emf">
+  <metadata
+     id="metadata3437">
+    <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>
+  <sodipodi:namedview
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1"
+     objecttolerance="10"
+     gridtolerance="10"
+     guidetolerance="10"
+     inkscape:pageopacity="0"
+     inkscape:pageshadow="2"
+     inkscape:window-width="1920"
+     inkscape:window-height="1178"
+     id="namedview3435"
+     showgrid="false"
+     inkscape:zoom="1.5267495"
+     inkscape:cx="424.83308"
+     inkscape:cy="249.60542"
+     inkscape:window-x="1592"
+     inkscape:window-y="-8"
+     inkscape:window-maximized="1"
+     inkscape:current-layer="svg3187" />
+  <defs
+     id="defs3189" />
+  <g
+     id="g3191">
+    <path
+       style="fill:#d7e4bd;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 329.53835 298.27431   C 329.53835 294.14828 332.87668 290.80995 337.00271 290.80995   L 470.92982 290.80995   C 475.05585 290.80995 478.39418 294.14828 478.39418 298.27431   L 478.39418 328.13175   C 478.39418 332.25777 475.05585 335.59611 470.92982 335.59611   L 337.00271 335.59611   C 332.87668 335.59611 329.53835 332.25777 329.53835 328.13175   z"
+       id="path3193" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 329.53835 298.27431   C 329.53835 294.14828 332.87668 290.80995 337.00271 290.80995   L 470.92982 290.80995   C 475.05585 290.80995 478.39418 294.14828 478.39418 298.27431   L 478.39418 328.13175   C 478.39418 332.25777 475.05585 335.59611 470.92982 335.59611   L 337.00271 335.59611   C 332.87668 335.59611 329.53835 332.25777 329.53835 328.13175   z"
+       id="path3195" />
+    <text
+       xml:space="preserve"
+       x="356.03617"
+       y="322.12208"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3197">Java API</text>
+    <path
+       style="fill:#8eb4e3;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 152.32545 417.479   C 152.32545 412.45275 156.37646 408.40174 161.3652 408.40174   L 767.1036 408.40174   C 772.09235 408.40174 776.14336 412.45275 776.14336 417.479   L 776.14336 453.67552   C 776.14336 458.66426 772.09235 462.71527 767.1036 462.71527   L 161.3652 462.71527   C 156.37646 462.71527 152.32545 458.66426 152.32545 453.67552   z"
+       id="path3199" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 152.32545 417.479   C 152.32545 412.45275 156.37646 408.40174 161.3652 408.40174   L 767.1036 408.40174   C 772.09235 408.40174 776.14336 412.45275 776.14336 417.479   L 776.14336 453.67552   C 776.14336 458.66426 772.09235 462.71527 767.1036 462.71527   L 161.3652 462.71527   C 156.37646 462.71527 152.32545 458.66426 152.32545 453.67552   z"
+       id="path3201" />
+    <text
+       xml:space="preserve"
+       x="386.56491"
+       y="444.47987"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3203">Flink Runtime</text>
+    <path
+       style="fill:#c6d9f1;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 251.57517 353.75063   C 251.57517 348.68687 255.62618 344.59835 260.68994 344.59835   L 767.02859 344.59835   C 772.09235 344.59835 776.14336 348.68687 776.14336 353.75063   L 776.14336 390.20971   C 776.14336 395.23596 772.09235 399.32448 767.02859 399.32448   L 260.68994 399.32448   C 255.62618 399.32448 251.57517 395.23596 251.57517 390.20971   z"
+       id="path3205" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 251.57517 353.75063   C 251.57517 348.68687 255.62618 344.59835 260.68994 344.59835   L 767.02859 344.59835   C 772.09235 344.59835 776.14336 348.68687 776.14336 353.75063   L 776.14336 390.20971   C 776.14336 395.23596 772.09235 399.32448 767.02859 399.32448   L 260.68994 399.32448   C 255.62618 399.32448 251.57517 395.23596 251.57517 390.20971   z"
+       id="path3207" />
+    <text
+       xml:space="preserve"
+       x="341.75921"
+       y="380.87575"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3209">Flink Common API / Optimizer</text>
+    <path
+       style="fill:#93cddd;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 491.14736 298.27431   C 491.14736 294.14828 494.48569 290.80995 498.61172 290.80995   L 611.28979 290.80995   C 615.41582 290.80995 618.75415 294.14828 618.75415 298.27431   L 618.75415 328.13175   C 618.75415 332.25777 615.41582 335.59611 611.28979 335.59611   L 498.61172 335.59611   C 494.48569 335.59611 491.14736 332.25777 491.14736 328.13175   z"
+       id="path3211" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 491.14736 298.27431   C 491.14736 294.14828 494.48569 290.80995 498.61172 290.80995   L 611.28979 290.80995   C 615.41582 290.80995 618.75415 294.14828 618.75415 298.27431   L 618.75415 328.13175   C 618.75415 332.25777 615.41582 335.59611 611.28979 335.59611   L 498.61172 335.59611   C 494.48569 335.59611 491.14736 332.25777 491.14736 328.13175   z"
+       id="path3213" />
+    <text
+       xml:space="preserve"
+       x="501.74312"
+       y="322.12208"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3215">Scala</text>
+    <text
+       xml:space="preserve"
+       x="569.86009"
+       y="322.12208"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3217">API</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 630.6071 298.31182   C 630.6071 294.18579 633.94543 290.80995 638.07146 290.80995   L 767.74126 290.80995   C 771.86729 290.80995 775.24313 294.18579 775.24313 298.31182   L 775.24313 328.28178   C 775.24313 332.44532 771.86729 335.78365 767.74126 335.78365   L 638.07146 335.78365   C 633.94543 335.78365 630.6071 332.44532 630.6071 328.28178   z"
+       id="path3219" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.037509345px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 629.33178 328.28178   L 629.33178 325.80617   L 631.84491 325.80617   L 631.84491 328.28178   z  M 629.33178 323.29304   L 629.33178 320.77991   L 631.84491 320.77991   L 631.84491 323.29304   z  M 629.33178 318.3043   L 629.33178 315.79117   L 631.84491 315.79117   L 631.84491 318.3043   z  M 629.33178 313.27805   L 629.33178 310.80243   L 631.84491 310.80243   L 631.84491 313.27805   z  M 629.33178 308.2893   L 629.33178 305.77618   L 631.84491 305.77618   L 631.84491 308.2893   z  M 629.33178 303.30056   L 629.33178 300.78743   L 631.84491 300.78743   L 631.84491 303.30056   z  M 629.33178 298.2368   L 629.36929 297.48661   L 629.51933 296.6239   L 629.70688 295.76118   L 629.85691 295.4236   L 632.18249 296.28631   L 632.14498 296.39884   L 631.99495 296.99899   L 631.88242 297.59914   L 631.84491 298.34933   z  M 631.28227 292.87296   L 631.8074 292.2353   C 631.88242 292.16028 631.91993 292.08527 631.99495 292.04776   L 633.12023 291.14753   C 633.15774 291.11002
  633.19524 291.07251 633.23275 291.035   L 633.4203 290.92248   L 634.73313 293.06051   L 634.54558 293.17304   L 634.69562 293.06051   L 633.57034 293.96073   L 633.75788 293.8107   L 633.23275 294.48586   z  M 636.04595 289.7972   L 636.27101 289.75969   L 637.13373 289.60965   L 638.03395 289.57214   L 638.85916 289.57214   L 638.85916 292.08527   L 638.14648 292.08527   L 637.50882 292.08527   L 636.90867 292.16028   L 636.68361 292.2353   z  M 641.33477 289.57214   L 643.8479 289.57214   L 643.8479 292.08527   L 641.33477 292.08527   z  M 646.36102 289.57214   L 648.83664 289.57214   L 648.83664 292.08527   L 646.36102 292.08527   z  M 651.34977 289.57214   L 653.86289 289.57214   L 653.86289 292.08527   L 651.34977 292.08527   z  M 656.33851 289.57214   L 658.85164 289.57214   L 658.85164 292.08527   L 656.33851 292.08527   z  M 661.36476 289.57214   L 663.84038 289.57214   L 663.84038 292.08527   L 661.36476 292.08527   z  M 666.35351 289.57214   L 668.86663 289.57214   L 668
 .86663 292.08527   L 666.35351 292.08527   z  M 671.34225 289.57214   L 673.85537 289.57214   L 673.85537 292.08527   L 671.34225 292.08527   z  M 676.3685 289.57214   L 678.84412 289.57214   L 678.84412 292.08527   L 676.3685 292.08527   z  M 681.35724 289.57214   L 683.87037 289.57214   L 683.87037 292.08527   L 681.35724 292.08527   z  M 686.34599 289.57214   L 688.85911 289.57214   L 688.85911 292.08527   L 686.34599 292.08527   z  M 691.37224 289.57214   L 693.84785 289.57214   L 693.84785 292.08527   L 691.37224 292.08527   z  M 696.36098 289.57214   L 698.87411 289.57214   L 698.87411 292.08527   L 696.36098 292.08527   z  M 701.34972 289.57214   L 703.86285 289.57214   L 703.86285 292.08527   L 701.34972 292.08527   z  M 706.37598 289.57214   L 708.85159 289.57214   L 708.85159 292.08527   L 706.37598 292.08527   z  M 711.36472 289.57214   L 713.87784 289.57214   L 713.87784 292.08527   L 711.36472 292.08527   z  M 716.35346 289.57214   L 718.86659 289.57214   L 718.86659 29
 2.08527   L 716.35346 292.08527   z  M 721.37971 289.57214   L 723.85533 289.57214   L 723.85533 292.08527   L 721.37971 292.08527   z  M 726.36846 289.57214   L 728.88158 289.57214   L 728.88158 292.08527   L 726.36846 292.08527   z  M 731.3572 289.57214   L 733.87033 289.57214   L 733.87033 292.08527   L 731.3572 292.08527   z  M 736.38345 289.57214   L 738.85907 289.57214   L 738.85907 292.08527   L 736.38345 292.08527   z  M 741.37219 289.57214   L 743.88532 289.57214   L 743.88532 292.08527   L 741.37219 292.08527   z  M 746.36094 289.57214   L 748.87406 289.57214   L 748.87406 292.08527   L 746.36094 292.08527   z  M 751.38719 289.57214   L 753.86281 289.57214   L 753.86281 292.08527   L 751.38719 292.08527   z  M 756.37593 289.57214   L 758.88906 289.57214   L 758.88906 292.08527   L 756.37593 292.08527   z  M 761.36468 289.57214   L 763.8778 289.57214   L 763.8778 292.08527   L 761.36468 292.08527   z  M 766.39093 289.57214   L 767.74126 289.57214   L 768.56647 289.60965   L
  769.05409 289.68467   L 768.679 292.16028   L 768.45394 292.12278   L 767.74126 292.08527   L 766.39093 292.08527   z  M 771.79227 290.54738   L 771.86729 290.58489   L 772.57997 291.035   C 772.65499 291.07251 772.6925 291.11002 772.73001 291.14753   L 773.85529 292.04776   C 773.8928 292.08527 773.96781 292.16028 774.00532 292.2353   L 774.19287 292.42285   L 772.24238 293.99824   L 772.09235 293.8107   L 772.24238 293.96073   L 771.15461 293.06051   L 771.30465 293.17304   L 770.77952 292.83545   L 770.7045 292.79794   z  M 775.73075 294.74843   L 775.76826 294.86096   L 776.06834 295.64865   L 776.29339 296.51137   L 776.44343 297.37408   L 776.44343 297.63665   L 773.96781 297.74918   L 773.96781 297.74918   L 773.8928 297.11152   L 773.74276 296.51137   L 773.5177 295.94873   L 773.44268 295.79869   z  M 776.48094 300.18728   L 776.48094 302.70041   L 774.00532 302.70041   L 774.00532 300.18728   z  M 776.48094 305.17603   L 776.48094 307.68915   L 774.00532 307.68915   L 774
 .00532 305.17603   z  M 776.48094 310.20228   L 776.48094 312.6779   L 774.00532 312.6779   L 774.00532 310.20228   z  M 776.48094 315.19102   L 776.48094 317.70415   L 774.00532 317.70415   L 774.00532 315.19102   z  M 776.48094 320.17976   L 776.48094 322.69289   L 774.00532 322.69289   L 774.00532 320.17976   z  M 776.48094 325.20602   L 776.48094 327.68163   L 774.00532 327.68163   L 774.00532 325.20602   z  M 776.21838 330.49483   L 776.10585 330.83242   L 775.84328 331.65762   L 775.46819 332.40781   L 775.05559 333.12049   L 772.91755 331.77015   L 773.21763 331.32004   L 773.48019 330.79491   L 773.70525 330.19476   L 773.78027 329.85718   z  M 773.06759 335.22101   L 772.73001 335.48358   L 771.97982 335.9712   L 771.19212 336.34629   L 770.51696 336.60886   L 769.65424 334.24577   L 770.14186 334.05822   L 770.66699 333.83317   L 771.15461 333.53309   L 771.45469 333.27053   z  M 767.62874 337.05897   L 765.11561 337.05897   L 765.11561 334.54584   L 767.62874 334.54584   
 z  M 762.60248 337.05897   L 760.12687 337.05897   L 760.12687 334.54584   L 762.60248 334.54584   z  M 757.61374 337.05897   L 755.10061 337.05897   L 755.10061 334.54584   L 757.61374 334.54584   z  M 752.625 337.05897   L 750.11187 337.05897   L 750.11187 334.54584   L 752.625 334.54584   z  M 747.59875 337.05897   L 745.12313 337.05897   L 745.12313 334.54584   L 747.59875 334.54584   z  M 742.61 337.05897   L 740.09688 337.05897   L 740.09688 334.54584   L 742.61 334.54584   z  M 737.62126 337.05897   L 735.10813 337.05897   L 735.10813 334.54584   L 737.62126 334.54584   z  M 732.59501 337.05897   L 730.11939 337.05897   L 730.11939 334.54584   L 732.59501 334.54584   z  M 727.60626 337.05897   L 725.09314 337.05897   L 725.09314 334.54584   L 727.60626 334.54584   z  M 722.61752 337.05897   L 720.1044 337.05897   L 720.1044 334.54584   L 722.61752 334.54584   z  M 717.59127 337.05897   L 715.11565 337.05897   L 715.11565 334.54584   L 717.59127 334.54584   z  M 712.60253 337.
 05897   L 710.0894 337.05897   L 710.0894 334.54584   L 712.60253 334.54584   z  M 707.61378 337.05897   L 705.10066 337.05897   L 705.10066 334.54584   L 707.61378 334.54584   z  M 702.58753 337.05897   L 700.11192 337.05897   L 700.11192 334.54584   L 702.58753 334.54584   z  M 697.59879 337.05897   L 695.08566 337.05897   L 695.08566 334.54584   L 697.59879 334.54584   z  M 692.61005 337.05897   L 690.09692 337.05897   L 690.09692 334.54584   L 692.61005 334.54584   z  M 687.58379 337.05897   L 685.10818 337.05897   L 685.10818 334.54584   L 687.58379 334.54584   z  M 682.59505 337.05897   L 680.08193 337.05897   L 680.08193 334.54584   L 682.59505 334.54584   z  M 677.60631 337.05897   L 675.09318 337.05897   L 675.09318 334.54584   L 677.60631 334.54584   z  M 672.58006 337.05897   L 670.10444 337.05897   L 670.10444 334.54584   L 672.58006 334.54584   z  M 667.59131 337.05897   L 665.07819 337.05897   L 665.07819 334.54584   L 667.59131 334.54584   z  M 662.60257 337.05897   L
  660.08944 337.05897   L 660.08944 334.54584   L 662.60257 334.54584   z  M 657.57632 337.05897   L 655.1007 337.05897   L 655.1007 334.54584   L 657.57632 334.54584   z  M 652.58758 337.05897   L 650.07445 337.05897   L 650.07445 334.54584   L 652.58758 334.54584   z  M 647.59883 337.05897   L 645.08571 337.05897   L 645.08571 334.54584   L 647.59883 334.54584   z  M 642.57258 337.05897   L 640.09696 337.05897   L 640.09696 334.54584   L 642.57258 334.54584   z  M 637.50882 337.02146   L 637.24625 337.02146   L 636.38354 336.87142   L 635.55833 336.64637   L 634.73313 336.3838   L 634.6206 336.30878   L 635.59584 334.02071   L 635.59584 334.02071   L 636.19599 334.24577   L 636.75863 334.39581   L 637.39629 334.50833   L 637.65886 334.50833   z  M 632.29502 334.80841   L 631.99495 334.54584   C 631.91993 334.50833 631.88242 334.47083 631.8074 334.39581   L 630.90717 333.27053   L 630.45706 332.55785   L 632.55759 331.24502   L 632.85766 331.69513   L 633.75788 332.82041   L 633.570
 34 332.63287   L 633.87041 332.85792   z  M 629.48182 329.74465   L 629.4068 329.25703   L 629.33178 328.3568   L 631.84491 328.24427   L 631.88242 328.88193   L 631.95744 329.36955   z"
+       id="path3221" />
+    <text
+       xml:space="preserve"
+       x="637.14501"
+       y="322.21058"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3223">SQL,Python</text>
+    <path
+       style="fill:#bfbfbf;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 60.164989 553.03778   C 60.164989 547.07379 64.966185 542.27259 70.93017 542.27259   L 765.37817 542.27259   C 771.34216 542.27259 776.14336 547.07379 776.14336 553.03778   L 776.14336 596.13601   C 776.14336 602.1 771.34216 606.90119 765.37817 606.90119   L 70.93017 606.90119   C 64.966185 606.90119 60.164989 602.1 60.164989 596.13601   z"
+       id="path3225" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 60.164989 553.03778   C 60.164989 547.07379 64.966185 542.27259 70.93017 542.27259   L 765.37817 542.27259   C 771.34216 542.27259 776.14336 547.07379 776.14336 553.03778   L 776.14336 596.13601   C 776.14336 602.1 771.34216 606.90119 765.37817 606.90119   L 70.93017 606.90119   C 64.966185 606.90119 60.164989 602.1 60.164989 596.13601   z"
+       id="path3227" />
+    <text
+       xml:space="preserve"
+       x="72.366258"
+       y="581.54762"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3229">Storage </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 349.39954 556.61992   C 349.39954 551.66868 353.41304 547.63643 358.36428 547.63643   L 442.81657 547.63643   C 447.78655 547.63643 451.80005 551.66868 451.80005 556.61992   L 451.80005 592.53512   C 451.80005 597.5051 447.78655 601.5186 442.81657 601.5186   L 358.36428 601.5186   C 353.41304 601.5186 349.39954 597.5051 349.39954 592.53512   z"
+       id="path3231" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 349.39954 556.61992   C 349.39954 551.66868 353.41304 547.63643 358.36428 547.63643   L 442.81657 547.63643   C 447.78655 547.63643 451.80005 551.66868 451.80005 556.61992   L 451.80005 592.53512   C 451.80005 597.5051 447.78655 601.5186 442.81657 601.5186   L 358.36428 601.5186   C 353.41304 601.5186 349.39954 597.5051 349.39954 592.53512   z"
+       id="path3233" />
+    <text
+       xml:space="preserve"
+       x="369.22367"
+       y="583.49811"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3235">HDFS </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 218.30439 556.61992   C 218.30439 551.66868 222.33664 547.63643 227.28787 547.63643   L 324.02447 547.63643   C 328.97571 547.63643 333.00796 551.66868 333.00796 556.61992   L 333.00796 592.53512   C 333.00796 597.5051 328.97571 601.5186 324.02447 601.5186   L 227.28787 601.5186   C 222.33664 601.5186 218.30439 597.5051 218.30439 592.53512   z"
+       id="path3237" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 218.30439 556.61992   C 218.30439 551.66868 222.33664 547.63643 227.28787 547.63643   L 324.02447 547.63643   C 328.97571 547.63643 333.00796 551.66868 333.00796 556.61992   L 333.00796 592.53512   C 333.00796 597.5051 328.97571 601.5186 324.02447 601.5186   L 227.28787 601.5186   C 222.33664 601.5186 218.30439 597.5051 218.30439 592.53512   z"
+       id="path3239" />
+    <text
+       xml:space="preserve"
+       x="246.82887"
+       y="569.99475"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3241">Local </text>
+    <text
+       xml:space="preserve"
+       x="250.42977"
+       y="597.00147"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3243">Files </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 464.92833 557.80146   C 464.92833 552.85023 468.94183 548.81797 473.89306 548.81797   L 558.06403 548.81797   C 563.03402 548.81797 567.04752 552.85023 567.04752 557.80146   L 567.04752 593.71666   C 567.04752 598.68665 563.03402 602.70015 558.06403 602.70015   L 473.89306 602.70015   C 468.94183 602.70015 464.92833 598.68665 464.92833 593.71666   z"
+       id="path3245" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 464.92833 557.80146   C 464.92833 552.85023 468.94183 548.81797 473.89306 548.81797   L 558.06403 548.81797   C 563.03402 548.81797 567.04752 552.85023 567.04752 557.80146   L 567.04752 593.71666   C 567.04752 598.68665 563.03402 602.70015 558.06403 602.70015   L 473.89306 602.70015   C 468.94183 602.70015 464.92833 598.68665 464.92833 593.71666   z"
+       id="path3247" />
+    <text
+       xml:space="preserve"
+       x="501.12089"
+       y="584.68335"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3249">S3</text>
+    <path
+       style="fill:#c3d69b;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 60.164989 483.00783   C 60.164989 477.04384 64.966185 472.20514 70.93017 472.20514   L 765.37817 472.20514   C 771.34216 472.20514 776.14336 477.04384 776.14336 483.00783   L 776.14336 526.10607   C 776.14336 532.03254 771.34216 536.87125 765.37817 536.87125   L 70.93017 536.87125   C 64.966185 536.87125 60.164989 532.03254 60.164989 526.10607   z"
+       id="path3251" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 60.164989 483.00783   C 60.164989 477.04384 64.966185 472.20514 70.93017 472.20514   L 765.37817 472.20514   C 771.34216 472.20514 776.14336 477.04384 776.14336 483.00783   L 776.14336 526.10607   C 776.14336 532.03254 771.34216 536.87125 765.37817 536.87125   L 70.93017 536.87125   C 64.966185 536.87125 60.164989 532.03254 60.164989 526.10607   z"
+       id="path3253" />
+    <text
+       xml:space="preserve"
+       x="72.366258"
+       y="500.404"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3255">Cluster</text>
+    <text
+       xml:space="preserve"
+       x="72.366258"
+       y="524.40998"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3257">Manager </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 404.91337 486.57122   C 404.91337 481.61998 408.92687 477.60648 413.87811 477.60648   L 557.10754 477.60648   C 562.07753 477.60648 566.09103 481.61998 566.09103 486.57122   L 566.09103 522.50517   C 566.09103 527.4564 562.07753 531.4699 557.10754 531.4699   L 413.87811 531.4699   C 408.92687 531.4699 404.91337 527.4564 404.91337 522.50517   z"
+       id="path3259" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 404.91337 486.57122   C 404.91337 481.61998 408.92687 477.60648 413.87811 477.60648   L 557.10754 477.60648   C 562.07753 477.60648 566.09103 481.61998 566.09103 486.57122   L 566.09103 522.50517   C 566.09103 527.4564 562.07753 531.4699 557.10754 531.4699   L 413.87811 531.4699   C 408.92687 531.4699 404.91337 527.4564 404.91337 522.50517   z"
+       id="path3261" />
+    <text
+       xml:space="preserve"
+       x="455.20194"
+       y="513.45725"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3263">YARN </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 581.6199 486.57122   C 581.6199 481.61998 585.6334 477.60648 590.58463 477.60648   L 756.15088 477.60648   C 761.10211 477.60648 765.11561 481.61998 765.11561 486.57122   L 765.11561 522.50517   C 765.11561 527.4564 761.10211 531.4699 756.15088 531.4699   L 590.58463 531.4699   C 585.6334 531.4699 581.6199 527.4564 581.6199 522.50517   z"
+       id="path3265" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 581.6199 486.57122   C 581.6199 481.61998 585.6334 477.60648 590.58463 477.60648   L 756.15088 477.60648   C 761.10211 477.60648 765.11561 481.61998 765.11561 486.57122   L 765.11561 522.50517   C 765.11561 527.4564 761.10211 531.4699 756.15088 531.4699   L 590.58463 531.4699   C 585.6334 531.4699 581.6199 527.4564 581.6199 522.50517   z"
+       id="path3267" />
+    <text
+       xml:space="preserve"
+       x="651.2216"
+       y="513.45725"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3269">EC2 </text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 230.5887 486.57122   C 230.5887 481.61998 234.62095 477.60648 239.57218 477.60648   L 384.35825 477.60648   C 389.30949 477.60648 393.34174 481.61998 393.34174 486.57122   L 393.34174 522.50517   C 393.34174 527.4564 389.30949 531.4699 384.35825 531.4699   L 239.57218 531.4699   C 234.62095 531.4699 230.5887 527.4564 230.5887 522.50517   z"
+       id="path3271" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 230.5887 486.57122   C 230.5887 481.61998 234.62095 477.60648 239.57218 477.60648   L 384.35825 477.60648   C 389.30949 477.60648 393.34174 481.61998 393.34174 486.57122   L 393.34174 522.50517   C 393.34174 527.4564 389.30949 531.4699 384.35825 531.4699   L 239.57218 531.4699   C 234.62095 531.4699 230.5887 527.4564 230.5887 522.50517   z"
+       id="path3273" />
+    <text
+       xml:space="preserve"
+       x="278.33392"
+       y="513.45725"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3275">Direct</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 698.16143 556.63867   C 698.16143 551.68744 702.21244 547.63643 707.16367 547.63643   L 756.15088 547.63643   C 761.10211 547.63643 765.11561 551.68744 765.11561 556.63867   L 765.11561 592.53512   C 765.11561 597.52386 761.10211 601.53736 756.15088 601.53736   L 707.16367 601.53736   C 702.21244 601.53736 698.16143 597.52386 698.16143 592.53512   z"
+       id="path3277" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 698.16143 556.63867   C 698.16143 551.68744 702.21244 547.63643 707.16367 547.63643   L 756.15088 547.63643   C 761.10211 547.63643 765.11561 551.68744 765.11561 556.63867   L 765.11561 592.53512   C 765.11561 597.52386 761.10211 601.53736 756.15088 601.53736   L 707.16367 601.53736   C 702.21244 601.53736 698.16143 597.52386 698.16143 592.53512   z"
+       id="path3279" />
+    <text
+       xml:space="preserve"
+       x="719.33685"
+       y="583.49811"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3281">...</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 581.6199 557.80146   C 581.6199 552.85023 585.6334 548.83673 590.58463 548.83673   L 674.7556 548.83673   C 679.70683 548.83673 683.72033 552.85023 683.72033 557.80146   L 683.72033 593.73541   C 683.72033 598.68665 679.70683 602.70015 674.7556 602.70015   L 590.58463 602.70015   C 585.6334 602.70015 581.6199 598.68665 581.6199 593.73541   z"
+       id="path3283" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 581.6199 557.80146   C 581.6199 552.85023 585.6334 548.83673 590.58463 548.83673   L 674.7556 548.83673   C 679.70683 548.83673 683.72033 552.85023 683.72033 557.80146   L 683.72033 593.73541   C 683.72033 598.68665 679.70683 602.70015 674.7556 602.70015   L 590.58463 602.70015   C 585.6334 602.70015 581.6199 598.68665 581.6199 593.73541   z"
+       id="path3285" />
+    <text
+       xml:space="preserve"
+       x="603.24956"
+       y="584.68335"
+       style="font-size:22.505608px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3287">JDBC</text>
+    <text
+       xml:space="preserve"
+       x="59.786351"
+       y="368.91759"
+       style="font-size:13.803439px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3289">JobGraphs</text>
+    <text
+       xml:space="preserve"
+       x="23.327268"
+       y="385.4217"
+       style="font-size:13.803439px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3291">(generic data stream</text>
+    <text
+       xml:space="preserve"
+       x="63.387248"
+       y="401.92581"
+       style="font-size:13.803439px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3293">program)</text>
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 86.496548 342.57284   C 86.496548 338.67187 89.666088 335.52109 93.548305 335.52109   C 97.449277 335.52109 100.60006 338.67187 100.60006 342.57284   C 100.60006 346.47382 97.449277 349.6246 93.548305 349.6246   C 89.666088 349.6246 86.496548 346.47382 86.496548 342.57284 "
+       id="path3295" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 86.496548 342.57284   C 86.496548 338.67187 89.666088 335.52109 93.548305 335.52109   C 97.449277 335.52109 100.60006 338.67187 100.60006 342.57284   C 100.60006 346.47382 97.449277 349.6246 93.548305 349.6246   C 89.666088 349.6246 86.496548 346.47382 86.496548 342.57284  "
+       id="path3297" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 123.8371 342.57284   C 123.8371 338.67187 126.98789 335.52109 130.88886 335.52109   C 134.77107 335.52109 137.92186 338.67187 137.92186 342.57284   C 137.92186 346.47382 134.77107 349.6246 130.88886 349.6246   C 126.98789 349.6246 123.8371 346.47382 123.8371 342.57284 "
+       id="path3299" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 123.8371 342.57284   C 123.8371 338.67187 126.98789 335.52109 130.88886 335.52109   C 134.77107 335.52109 137.92186 338.67187 137.92186 342.57284   C 137.92186 346.47382 134.77107 349.6246 130.88886 349.6246   C 126.98789 349.6246 123.8371 346.47382 123.8371 342.57284  "
+       id="path3301" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 106.62031 315.41608   C 106.62031 311.51511 109.82736 308.36432 113.76584 308.36432   C 117.72308 308.36432 120.93013 311.51511 120.93013 315.41608   C 120.93013 319.2983 117.72308 322.46783 113.76584 322.46783   C 109.82736 322.46783 106.62031 319.2983 106.62031 315.41608 "
+       id="path3303" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 106.62031 315.41608   C 106.62031 311.51511 109.82736 308.36432 113.76584 308.36432   C 117.72308 308.36432 120.93013 311.51511 120.93013 315.41608   C 120.93013 319.2983 117.72308 322.46783 113.76584 322.46783   C 109.82736 322.46783 106.62031 319.2983 106.62031 315.41608  "
+       id="path3305" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 106.61093 287.40598   C 106.61093 283.51438 109.81798 280.3636 113.76584 280.3636   C 117.72308 280.3636 120.92075 283.51438 120.92075 287.40598   C 120.92075 291.30695 117.72308 294.45773 113.76584 294.45773   C 109.81798 294.45773 106.61093 291.30695 106.61093 287.40598 "
+       id="path3307" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 106.61093 287.40598   C 106.61093 283.51438 109.81798 280.3636 113.76584 280.3636   C 117.72308 280.3636 120.92075 283.51438 120.92075 287.40598   C 120.92075 291.30695 117.72308 294.45773 113.76584 294.45773   C 109.81798 294.45773 106.61093 291.30695 106.61093 287.40598  "
+       id="path3309" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 97.974408 337.04022   L 112.07792 322.9367   L 113.18445 324.04323   L 99.080934 338.14674   z  M 106.71409 324.06198   L 113.67207 322.44908   L 112.05917 329.42582   C 111.96539 329.83842 111.55279 330.10099 111.12143 330.00721   C 110.70883 329.91344 110.44626 329.48208 110.54004 329.06948   L 111.87162 323.33055   L 112.80935 324.26828   L 107.05167 325.58111   C 106.63907 325.67488 106.22646 325.41232 106.11394 324.99972   C 106.02016 324.58711 106.28273 324.15576 106.71409 324.06198   z"
+       id="path3311" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 112.88437 308.55187   L 113.07192 296.13627   L 114.62856 296.15503   L 114.44101 308.57062   z  M 109.99615 300.67491   L 113.87837 294.67341   L 117.57304 300.78743   C 117.7981 301.16253 117.68557 301.63139 117.31047 301.85645   C 116.93538 302.08151 116.46651 301.96898 116.24146 301.59388   L 113.18445 296.54888   L 114.51603 296.56763   L 111.30898 301.51887   C 111.08392 301.8752 110.5963 301.98773 110.22121 301.74392   C 109.86487 301.51887 109.7711 301.03124 109.99615 300.67491   z"
+       id="path3313" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 67.83565 342.57284   C 67.83565 338.67187 71.005189 335.52109 74.887406 335.52109   C 78.788378 335.52109 81.939163 338.67187 81.939163 342.57284   C 81.939163 346.47382 78.788378 349.6246 74.887406 349.6246   C 71.005189 349.6246 67.83565 346.47382 67.83565 342.57284 "
+       id="path3315" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 67.83565 342.57284   C 67.83565 338.67187 71.005189 335.52109 74.887406 335.52109   C 78.788378 335.52109 81.939163 338.67187 81.939163 342.57284   C 81.939163 346.47382 78.788378 349.6246 74.887406 349.6246   C 71.005189 349.6246 67.83565 346.47382 67.83565 342.57284  "
+       id="path3317" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 49.174751 342.57284   C 49.174751 338.67187 52.325536 335.52109 56.226507 335.52109   C 60.127479 335.52109 63.278264 338.67187 63.278264 342.57284   C 63.278264 346.47382 60.127479 349.6246 56.226507 349.6246   C 52.325536 349.6246 49.174751 346.47382 49.174751 342.57284 "
+       id="path3319" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 49.174751 342.57284   C 49.174751 338.67187 52.325536 335.52109 56.226507 335.52109   C 60.127479 335.52109 63.278264 338.67187 63.278264 342.57284   C 63.278264 346.47382 60.127479 349.6246 56.226507 349.6246   C 52.325536 349.6246 49.174751 346.47382 49.174751 342.57284  "
+       id="path3321" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 30.513852 342.57284   C 30.513852 338.67187 33.664637 335.52109 37.565609 335.52109   C 41.46658 335.52109 44.617365 338.67187 44.617365 342.57284   C 44.617365 346.47382 41.46658 349.6246 37.565609 349.6246   C 33.664637 349.6246 30.513852 346.47382 30.513852 342.57284 "
+       id="path3323" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 30.513852 342.57284   C 30.513852 338.67187 33.664637 335.52109 37.565609 335.52109   C 41.46658 335.52109 44.617365 338.67187 44.617365 342.57284   C 44.617365 346.47382 41.46658 349.6246 37.565609 349.6246   C 33.664637 349.6246 30.513852 346.47382 30.513852 342.57284  "
+       id="path3325" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 69.298514 315.41608   C 69.298514 311.51511 72.486808 308.36432 76.444044 308.36432   C 80.40128 308.36432 83.608329 311.51511 83.608329 315.41608   C 83.608329 319.2983 80.40128 322.46783 76.444044 322.46783   C 72.486808 322.46783 69.298514 319.2983 69.298514 315.41608 "
+       id="path3327" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 69.298514 315.41608   C 69.298514 311.51511 72.486808 308.36432 76.444044 308.36432   C 80.40128 308.36432 83.608329 311.51511 83.608329 315.41608   C 83.608329 319.2983 80.40128 322.46783 76.444044 322.46783   C 72.486808 322.46783 69.298514 319.2983 69.298514 315.41608  "
+       id="path3329" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 142.498 342.57284   C 142.498 338.67187 145.64878 335.52109 149.54976 335.52109   C 153.43197 335.52109 156.60151 338.67187 156.60151 342.57284   C 156.60151 346.47382 153.43197 349.6246 149.54976 349.6246   C 145.64878 349.6246 142.498 346.47382 142.498 342.57284 "
+       id="path3331" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5566378px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 142.498 342.57284   C 142.498 338.67187 145.64878 335.52109 149.54976 335.52109   C 153.43197 335.52109 156.60151 338.67187 156.60151 342.57284   C 156.60151 346.47382 153.43197 349.6246 149.54976 349.6246   C 145.64878 349.6246 142.498 346.47382 142.498 342.57284  "
+       id="path3333" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 69.289137 287.40598   C 69.289137 283.51438 72.486808 280.3636 76.444044 280.3636   C 80.391903 280.3636 83.598952 283.51438 83.598952 287.40598   C 83.598952 291.30695 80.391903 294.45773 76.444044 294.45773   C 72.486808 294.45773 69.289137 291.30695 69.289137 287.40598 "
+       id="path3335" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 69.289137 287.40598   C 69.289137 283.51438 72.486808 280.3636 76.444044 280.3636   C 80.391903 280.3636 83.598952 283.51438 83.598952 287.40598   C 83.598952 291.30695 80.391903 294.45773 76.444044 294.45773   C 72.486808 294.45773 69.289137 291.30695 69.289137 287.40598  "
+       id="path3337" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 74.643596 334.78965   L 112.02166 322.18651   L 112.52803 323.66813   L 75.149972 336.27127   z  M 106.71409 320.79867   L 113.67207 322.44908   L 109.13344 327.98171   C 108.87087 328.31929 108.3645 328.3568 108.04567 328.09424   C 107.70808 327.81292 107.65182 327.32529 107.93314 326.98771   L 111.66532 322.43033   L 112.09668 323.68689   L 106.35775 322.3178   C 105.94514 322.22402 105.68258 321.79267 105.77635 321.38006   C 105.87012 320.96746 106.30148 320.7049 106.71409 320.79867   z"
+       id="path3339" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 126.18143 336.87142   L 83.176971 320.17976   L 82.614331 321.64263   L 125.61879 338.31553   z  M 88.540808 319.03573   L 81.52656 320.38607   L 85.802626 326.10624   C 86.065191 326.44383 86.552812 326.51884 86.890397 326.25628   C 87.246735 326.01247 87.321754 325.52485 87.059189 325.16851   L 83.514556 320.44233   L 83.045689 321.68014   L 88.840882 320.57361   C 89.253485 320.49859 89.534805 320.08599 89.459787 319.67339   C 89.384768 319.24203 88.972165 318.96071 88.540808 319.03573   z"
+       id="path3341" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 144.99237 336.94644   L 120.49877 320.55486   L 119.63605 321.84893   L 144.12966 338.24051   z  M 125.99389 320.61112   L 118.84836 320.38607   L 121.77409 326.89394   C 121.96163 327.28779 122.41175 327.47533 122.80559 327.28779   C 123.19944 327.11899 123.38699 326.65013 123.19944 326.25628   L 120.78009 320.87369   L 120.04866 321.98021   L 125.93762 322.16776   C 126.36898 322.18651 126.74408 321.84893 126.74408 321.41757   C 126.76283 320.98622 126.42525 320.61112 125.99389 320.61112   z"
+       id="path3343" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 55.682622 334.9772   L 69.767381 320.87369   L 70.873906 321.98021   L 56.789148 336.08373   z  M 64.403545 321.98021   L 71.361528 320.38607   L 69.767381 327.34405   C 69.673607 327.77541 69.24225 328.03797 68.829647 327.92544   C 68.39829 327.83167 68.135724 327.41907 68.248252 326.98771   L 69.561079 321.24878   L 70.498813 322.18651   L 64.759883 323.5181   C 64.328526 323.61187 63.915923 323.3493 63.82215 322.91795   C 63.728376 322.50534 63.990942 322.09274 64.403545 321.98021   z"
+       id="path3345" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 37.246779 334.80841   L 69.711117 320.27354   L 70.348776 321.69889   L 37.884438 336.23376   z  M 64.291017 319.41082   L 71.380283 320.38607   L 67.385537 326.31254   C 67.141727 326.66888 66.654105 326.78141 66.297766 326.5376   C 65.941428 326.29379 65.847654 325.80617 66.091465 325.44983   L 69.373533 320.55486   L 69.917418 321.75516   L 64.084715 320.94871   C 63.653358 320.89244 63.353283 320.49859 63.409547 320.06724   C 63.465811 319.63588 63.859659 319.3358 64.291017 319.41082   z"
+       id="path3347" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 75.562575 308.55187   L 75.750121 296.13627   L 77.306759 296.15503   L 77.119212 308.57062   z  M 72.674355 300.67491   L 76.556572 294.67341   L 80.251243 300.78743   C 80.476299 301.16253 80.345016 301.63139 79.988677 301.85645   C 79.613584 302.08151 79.125962 301.96898 78.919661 301.59388   L 78.919661 301.59388   L 75.862649 296.54888   L 77.175476 296.56763   L 73.987182 301.51887   C 73.743371 301.8752 73.274505 301.98773 72.899411 301.74392   C 72.543072 301.51887 72.430544 301.03124 72.674355 300.67491   z"
+       id="path3349" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 81.095203 309.78968   L 107.03291 292.55413   L 107.89563 293.8482   L 81.957918 311.08375   z  M 101.55655 292.59164   L 108.70208 292.38534   L 105.73884 298.89321   C 105.57005 299.28706 105.10118 299.45585 104.70734 299.28706   C 104.31349 299.09951 104.1447 298.6494 104.31349 298.25555   L 106.75159 292.87296   L 107.48303 293.97949   L 101.59406 294.14828   C 101.1627 294.16703 100.80636 293.82945 100.78761 293.39809   C 100.78761 292.96674 101.12519 292.59164 101.55655 292.59164   z"
+       id="path3351" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 109.11468 309.78968   L 83.176971 292.55413   L 82.314257 293.8482   L 108.25197 311.08375   z  M 88.67209 292.59164   L 81.52656 292.38534   L 84.471044 298.89321   C 84.658591 299.28706 85.108703 299.45585 85.502551 299.28706   C 85.896399 299.09951 86.083946 298.6494 85.896399 298.25555   L 85.896399 298.25555   L 83.458292 292.87296   L 82.726859 293.97949   L 88.615826 294.14828   C 89.047184 294.16703 89.422277 293.82945 89.422277 293.39809   C 89.441032 292.96674 89.103448 292.59164 88.67209 292.59164   z"
+       id="path3353" />
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 383.64558 205.23238   L 426.18117 205.23238   L 426.18117 247.76798   L 419.09191 254.85724   L 383.64558 254.85724  z "
+       id="path3355" />
+    <path
+       style="fill:#cdcdcd;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 419.09191 254.85724   L 420.49851 249.19333   L 426.18117 247.76798  z "
+       id="path3357" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.2565631px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 419.09191 254.85724   L 420.49851 249.19333   L 426.18117 247.76798   L 419.09191 254.85724   L 383.64558 254.85724   L 383.64558 205.23238   L 426.18117 205.23238   L 426.18117 247.76798  "
+       id="path3359" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 209.75225   L 415.54727 209.75225  "
+       id="path3361" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 214.25338   L 420.85485 214.25338  "
+       id="path3363" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 218.77325   L 410.22095 218.77325  "
+       id="path3365" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 223.27437   L 420.85485 223.27437  "
+       id="path3367" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 227.79425   L 420.85485 227.79425  "
+       id="path3369" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.95315 232.31413   L 402.25021 232.31413  "
+       id="path3371" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 388.99066 236.81525   L 415.54727 236.81525  "
+       id="path3373" />
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 532.10756 204.29464   L 574.64316 204.29464   L 574.64316 246.81149   L 567.55389 253.90075   L 532.10756 253.90075  z "
+       id="path3375" />
+    <path
+       style="fill:#cdcdcd;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 567.55389 253.90075   L 568.97925 248.23684   L 574.64316 246.81149  z "
+       id="path3377" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.2565631px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 567.55389 253.90075   L 568.97925 248.23684   L 574.64316 246.81149   L 567.55389 253.90075   L 532.10756 253.90075   L 532.10756 204.29464   L 574.64316 204.29464   L 574.64316 246.81149  "
+       id="path3379" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 208.79577   L 564.00926 208.79577  "
+       id="path3381" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 213.31564   L 569.33559 213.31564  "
+       id="path3383" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 217.81676   L 558.70169 217.81676  "
+       id="path3385" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 222.33664   L 569.33559 222.33664  "
+       id="path3387" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 226.83776   L 569.33559 226.83776  "
+       id="path3389" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.43389 231.35764   L 550.73095 231.35764  "
+       id="path3391" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:1.8754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 537.45264 235.85876   L 564.00926 235.85876  "
+       id="path3393" />
+    <text
+       xml:space="preserve"
+       x="357.696"
+       y="272.13059"
+       style="font-size:13.803439px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3395">Java Program</text>
+    <text
+       xml:space="preserve"
+       x="505.39607"
+       y="273.082"
+       style="font-size:13.803439px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3397">Scala Program</text>
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 196.11761 240.36926   L 196.11761 248.03054   L 216.53207 248.03054   L 216.53207 240.36926   L 196.11761 240.36926  z "
+       id="path3399" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 196.11761 240.36926   L 216.53207 240.36926   L 216.53207 248.03054   L 196.11761 248.03054  z "
+       id="path3401" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 236.94653 240.36926   L 236.94653 248.03054   L 257.36099 248.03054   L 257.36099 240.36926   L 236.94653 240.36926  z "
+       id="path3403" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 236.94653 240.36926   L 257.36099 240.36926   L 257.36099 248.03054   L 236.94653 248.03054  z "
+       id="path3405" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 216.53207 223.37752   L 216.53207 231.02943   L 236.94653 231.02943   L 236.94653 223.37752   L 216.53207 223.37752  z "
+       id="path3407" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 216.53207 223.37752   L 236.94653 223.37752   L 236.94653 231.02943   L 216.53207 231.02943  z "
+       id="path3409" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 216.53207 207.02345   L 216.53207 214.68473   L 236.94653 214.68473   L 236.94653 207.02345   L 216.53207 207.02345  z "
+       id="path3411" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 216.53207 207.02345   L 236.94653 207.02345   L 236.94653 214.68473   L 216.53207 214.68473  z "
+       id="path3413" />
+    <path
+       style="fill:#9bbb59;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 196.11761 257.76422   L 196.11761 265.41612   L 216.53207 265.41612   L 216.53207 257.76422   L 196.11761 257.76422  z "
+       id="path3415" />
+    <path
+       style="fill:none;stroke:#71893f;stroke-width:1.5660151px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 196.11761 257.76422   L 216.53207 257.76422   L 216.53207 265.41612   L 196.11761 265.41612  z "
+       id="path3417" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.0093773361px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 205.54183 257.8955   L 205.54183 252.96302   C 205.54183 252.53166 205.88879 252.17532 206.32015 252.17532   L 206.32015 252.17532   L 205.54183 252.96302   L 205.54183 249.50278   L 207.10785 249.50278   L 207.10785 252.96302   C 207.10785 253.39438 206.75151 253.74134 206.32015 253.74134   L 206.32015 253.74134   L 207.10785 252.96302   L 207.10785 257.8955   z  M 202.53171 254.0883   L 206.32015 248.02116   L 210.10859 254.0883   C 210.34303 254.45402 210.2305 254.93226 209.86478 255.16669   C 209.49907 255.39175 209.01144 255.27922 208.78639 254.91351   L 205.66374 249.91539   L 206.98594 249.91539   L 203.86329 254.91351   C 203.62885 255.27922 203.15061 255.39175 202.78489 255.16669   C 202.41918 254.93226 202.30665 254.45402 202.53171 254.0883   z"
+       id="path3419" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.0093773361px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 247.93677 240.36926   L 247.93677 237.84675   C 247.93677 237.4154 247.58981 237.06843 247.15845 237.06843   L 226.74399 237.06843   L 227.52231 237.84675   L 227.52231 232.50167   L 225.95629 232.50167   L 225.95629 237.84675   C 225.95629 238.27811 226.31263 238.62507 226.74399 238.62507   L 247.15845 238.62507   L 246.38013 237.84675   L 246.38013 240.36926   z  M 230.53243 237.08719   L 226.74399 231.02943   L 222.95554 237.08719   C 222.72111 237.45291 222.83364 237.93115 223.19935 238.16558   C 223.56507 238.39064 224.05269 238.27811 224.27775 237.91239   L 227.4004 232.91427   L 226.0782 232.91427   L 229.20085 237.91239   C 229.43528 238.27811 229.91353 238.39064 230.27924 238.16558   C 230.64496 237.93115 230.75749 237.45291 230.53243 237.08719   z"
+       id="path3421" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.0093773361px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 205.54183 240.37863   L 205.54183 237.31225   C 205.54183 236.88089 205.88879 236.52455 206.32015 236.52455   L 226.74399 236.52455   L 225.95629 237.31225   L 225.95629 232.50167   L 227.52231 232.50167   L 227.52231 237.31225   C 227.52231 237.7436 227.17535 238.09056 226.74399 238.09056   L 206.32015 238.09056   L 207.10785 237.31225   L 207.10785 240.37863   z  M 222.95554 237.08719   L 226.74399 231.02943   L 230.53243 237.08719   C 230.75749 237.45291 230.64496 237.93115 230.27924 238.16558   C 229.91353 238.39064 229.43528 238.27811 229.20085 237.91239   L 226.0782 232.91427   L 227.4004 232.91427   L 224.27775 237.91239   C 224.05269 238.27811 223.56507 238.39064 223.19935 238.16558   C 222.83364 237.93115 222.72111 237.45291 222.95554 237.08719   z"
+       id="path3423" />
+    <path
+       style="fill:#77933c;fill-rule:nonzero;fill-opacity:1;stroke:#77933c;stroke-width:0.0093773361px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 225.95629 223.37752   L 225.95629 219.02644   C 225.95629 218.59508 226.31263 218.24812 226.74399 218.24812   L 226.74399 218.24812   L 225.95629 219.02644   L 225.95629 216.1476   L 227.52231 216.1476   L 227.52231 219.02644   C 227.52231 219.4578 227.17535 219.81414 226.74399 219.81414   L 226.74399 219.81414   L 227.52231 219.02644   L 227.52231 223.37752   z  M 222.95554 220.73312   L 226.74399 214.67536   L 230.53243 220.73312   C 230.75749 221.10821 230.64496 221.58645 230.27924 221.81151   C 229.91353 222.04594 229.43528 221.93341 229.20085 221.5677   L 226.0782 216.5602   L 227.4004 216.5602   L 224.27775 221.5677   C 224.05269 221.93341 223.56507 222.04594 223.19935 221.81151   C 222.83364 221.58645 222.72111 221.10821 222.95554 220.73312   z"
+       id="path3425" />
+    <text
+       xml:space="preserve"
+       x="169.74848"
+       y="282.5322"
+       style="font-size:13.803439px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3427">Operator DAG</text>
+    <text
+       xml:space="preserve"
+       x="165.69747"
+       y="299.03631"
+       style="font-size:13.803439px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3429">(type agnostic)</text>
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 265.73495 283.42061   L 293.17304 283.42061   C 293.86696 283.42061 294.4296 283.98325 294.4296 284.67717   L 294.4296 321.96146   L 291.91647 321.96146   L 291.91647 284.67717   L 293.17304 285.93373   L 265.73495 285.93373   z  M 298.63065 315.09725   L 293.17304 324.43708   L 287.71543 315.09725   C 287.35909 314.4971 287.56539 313.72816 288.16554 313.39057   C 288.76569 313.03423 289.53463 313.24054 289.87221 313.84069   L 294.24205 321.34255   L 292.08527 321.34255   L 296.47386 313.84069   C 296.81144 313.24054 297.58038 313.03423 298.18053 313.39057   C 298.78068 313.72816 298.96823 314.4971 298.63065 315.09725   z"
+       id="path3431" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 164.14089 336.87142   L 206.32015 336.87142   C 207.01407 336.87142 207.57671 337.43406 207.57671 338.12799   L 207.57671 391.74759   L 205.08234 391.74759   L 205.08234 338.12799   L 206.32015 339.38455   L 164.14089 339.38455   z  M 211.77776 384.88338   L 206.32015 394.24197   L 200.88129 384.88338   C 200.52496 384.28323 200.73126 383.51429 201.31265 383.17671   C 201.9128 382.82037 202.68174 383.02667 203.03808 383.62682   L 207.40792 391.12869   L 205.25113 391.12869   L 209.62097 383.62682   C 209.97731 383.02667 210.74625 382.82037 211.32765 383.17671   C 211.9278 383.51429 212.1341 384.28323 211.77776 384.88338   z"
+       id="path3433" />
+  </g>
+</svg>


[3/5] git commit: Add "How to contribute" and "coding guidelines" to the project.

Posted by se...@apache.org.
Add "How to contribute" and "coding guidelines" to the project.

[ci skip]


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

Branch: refs/heads/master
Commit: 05ff3339c5f45f8db8062b2a7dd3397f0ec66353
Parents: 3c5feae
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 27 17:41:22 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Jun 27 21:22:14 2014 +0200

----------------------------------------------------------------------
 docs/coding_guidelines.md | 65 +++++++++++++++++++++++++++++++
 docs/how_to_contribute.md | 86 ++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 151 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/05ff3339/docs/coding_guidelines.md
----------------------------------------------------------------------
diff --git a/docs/coding_guidelines.md b/docs/coding_guidelines.md
new file mode 100644
index 0000000..6ab5475
--- /dev/null
+++ b/docs/coding_guidelines.md
@@ -0,0 +1,65 @@
+---
+title:  "Coding Guidelines"
+---
+
+These are the coding and style guidelines that we follow in the Flink project.
+
+## Guidelines for pull requests and patches
+
+- A pull request should relate to a JIRA issue; create an issue if none exists for the change you want to make. The latest commit message should reference that issue. An example commit message would be "[FLINK-633] Fix NullPointerException for empty UDF parameters". That way, the pull request automatically gives a description of what it does, for example what bug does it fix in what way?
+
+- We consider pull requests as requests to merge the referenced code *as is* into the current *stable* master branch. Therefore, a pull request should not be "work in progress". Open a pull request if you are confident that it can be merged into the current master branch without problems. If you rather want comments on your code, post a link to your working branch.
+
+- Please do not combine various unrelated changes in a single pull request. Rather, open multiple individual pull requests. This ensures that pull requests are *topic related*, can be merged more easily, and typically result in topic-specific merge conflicts only.
+
+- Any pull request where the tests do not pass or which does not compile will not undergo any further review. We recommend to connect your private GitHub accounts with [Travis CI](http://travis-ci.org/) (like the Flink GitHub repository). Travis will run tests for all tested environments whenever you push something into *your* Github repository.
+
+- Please keep reformatting of source files to a minimum. Diffs become unreadable if you (or your IDE automatically) remove or replace whitespaces, reformat code, or comments. Also, other patches that affect the same files become un-mergeable. Please configure your IDE such that code is not automatically reformatted. Pull requests with excessive or unnecessary code reformatting might be rejected.
+
+- All new features need to be backed by tests, *strictly*. It is very easy that a later merge accidentally throws out a feature or breaks it. This will not be caught if the feature is not guarded by tests. Anything not covered by a test is considered cosmetic.
+
+- Before opening a pull request follow this checklist:
+ - Rebase onto the latest version of the master branch
+ - Clean up your commits, i.e., squash them in a reasonable way and give meaningful commit messages
+ - Run *all* tests either locally with ```mvn clean verify``` or use Travis CI to check the build
+
+- When you get comments on the pull request asking for changes, append commits for these changes. *Do not rebase and squash them.* It allows people to review the cleanup work independently. Otherwise reviewers have to go through the entire set of diffs again.
+
+- Public methods and classes that are part of the user-facing API need to have JavaDocs. Please write meaningful docs. Good docs are concise and informative.
+
+- Give meaningful exception messages. Try to imagine why an exception could be thrown (what a user did wrong) and give a message that will help a user to resolve the problem.
+
+- Follow the checkstyle rules (see below). The checkstyle plugin verifies these rules when you build the code. If your code does not follow the checkstyle rules, Maven will not compile it and consequently the build will fail.
+
+
+## Coding Style Guidelines
+
+- Make sure you have Apache License headers in your files. The RAT plugin is checking for that when you build the code.
+
+- We are using tabs for indentation, not spaces. We are not religious there, it just happened to be the way that we started with tabs, and it is important to not mix them (merge/diff conflicts).
+
+- All statements after `if`, `for`, `while`, `do`, ... must always be encapsulated in a block with curly braces (even if the block contains one statement):
+```
+for (...) {
+ ...
+}
+```
+If you are wondering why, recall the famous [*goto bug*](https://www.imperialviolet.org/2014/02/22/applebug.html) in Apple's SSL library.
+
+
+-  Do not use wildcard imports in the core files. They can cause problems when adding to the code and in some cases even during refactoring. Exceptions are the Tuple classes, Tuple-related utilities, and Flink user programs, when importing operators/functions. Tests are a special case of the user programs.
+  
+- Remove all unused imports.
+
+- Do not use raw generic types, unless strictly necessary (sometime necessary for signature matches, arrays).
+
+- Add annotations to suppress warnings, if they cannot be avoided (such as "unchecked", or "serial")
+
+- Do not add "non-javadoc" comments. 
+
+- Add comments to your code. What is it doing? Add JavaDocs or inherit them by not adding any comments to the methods. Do not automatically generate comments and avoid unnecessary comments like
+```
+i++;  // increment by one
+```
+
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/05ff3339/docs/how_to_contribute.md
----------------------------------------------------------------------
diff --git a/docs/how_to_contribute.md b/docs/how_to_contribute.md
new file mode 100644
index 0000000..ef23c38
--- /dev/null
+++ b/docs/how_to_contribute.md
@@ -0,0 +1,86 @@
+---
+title:  "Contribute"
+---
+
+The Flink project welcomes all sorts contributions in the form of code (improvements, features, bugfixes), documentation, tests, and community participation (discussions & questions).
+
+
+
+## Easy Issues for Starters
+
+We maintain all known issues and feature drafts in the [Flink project JIRA](https://issues.apache.org/jira/browse/FLINK-989?jql=project%20%3D%20FLINK).
+
+We also try to maintain a [list of simple "starter issues"](https://issues.apache.org/jira/browse/FLINK-933?jql=project%20%3D%20FLINK%20AND%20labels%20%3D%20starter) that we believe are good tasks for new contributors. Those tasks are meant to allow people to get into the project and become familiar with the process of contributing. Feel free to ask questions about issues that you would be interested in working on.
+
+In addition, you can find a list of ideas for projects and improvements in the [projects wiki page](http://the/wiki/url).
+
+
+
+## Contributing Code & Documentation
+
+This section gives you a brief introduction in how to contribute code and documentation to Flink. We maintain both the code and the documentation in the same repository, so the process is essentially the same for both. We use [git](http://git-scm.com/) for the code and documentation version control.
+
+The Flink project accepts code contributions though the [GitHub Mirror](https://github.com/apache/incubator-flink), in the form of [Pull Requests](https://help.github.com/articles/using-pull-requests). Pull requests are basically a simpler way of offering a patch, by providing a pointer to a code branch that contains the change.
+
+
+### Setting up the Infrastructure and Creating a Pull Request
+
+1. The first step is to create yourself a copy of the Flink code base. We suggest to fork the [Flink GitHub Mirror Repository](https://github.com/apache/incubator-flink) into your own [GitHub](https://github.com) account. You need to register on GitHub for that, if you have no account so far.
+
+2. Next, clone your repository fork to your local machine.
+```
+git clone https://github.com/<your-user-name>/incubator-flink.git
+```
+
+3. It is typically helpful to switch to a *topic branch* for the changes. To create a dedicated branch based on the current master, use the following command:
+```
+git checkout -b myBranch master
+```
+
+4. Now you can create your changes, compile the code, and validate the changes. Here are some pointers on how to [set up the Eclipse IDE for development](https://github.com/apache/incubator-flink/#eclipse-setup-and-debugging), and how to [build the code](https://github.com/apache/incubator-flink/#build-stratosphere).
+
+5. After you have finalized your contribution, verify the compliance with the contribution guidelines (see below), and commit them. To make the changes easily mergeable, please rebase them to the latest version of the main repositories master branch. Assuming you created a topic branch (step 3), you can follow this sequence of commands to do that:
+Switch to the master branch, update it to the latest revision, switch back to your topic branch, and rebase it on top of the master branch.
+```
+git checkout master
+git pull https://github.com/apache/incubator-flink.git master
+git checkout myBranch
+git rebase master
+```
+Have a look [here](https://help.github.com/articles/using-git-rebase) for more information about rebasing commits.
+
+
+6. Push the contribution it back into your fork of the Flink repository.
+```
+git push origin myBranch
+```
+Go the website of your repository fork (`https://github.com/<your-user-name>/incubator-flink`) and use the "Create Pull Request" button to start creating a pull request. Make sure that the base fork is `apache/incubator-flink master` and the head fork selects the branch with your changes. Give the pull request a meaningful description and send it.
+
+
+### Verifying the Compliance of your Code
+
+Before sending a patch or pull request, please verify that it complies with the guidelines of the project. While we try to keep the set of guidelines small and easy, it is important to follow those rules in order to guarantee good code quality, to allow efficient reviews, and to allow committers to easily merge your changes.
+
+Please have a look at the [coding guidelines](coding_guidelines.html) for a guide to the format of code and pull requests.
+
+Most important of all, verify that your changes are correct and do not break existing functionality. Run the existing tests by calling `mvn verify` in the root directory of the repository, and make sure that the tests succeed. We encourage every contributor to use a *continuous integration* service that will automatically test the code in your repository whenever you push a change. Flink is pre-configured for [Travis CI](http://docs.travis-ci.com/), which can be easily enabled for your private repository fork (it uses GitHub for authentication, so you so not need an additional account). Simply add the *Travis CI* hook to your repository (*settings --> webhooks & services --> add service*) and enable tests for the "incubator-flink" repository on [Travis](https://travis-ci.org/profile).
+
+When contributing documentation, please review the rendered HTML versions of the documents you changed. You can look at the HTML pages by using the rendering script in preview mode. 
+```
+cd docs
+./build_docs.sh -p
+```
+Now, open your browser at `http://localhost:4000` and check out the pages you changed.
+
+
+
+## How to become a committer
+
+There is no strict protocol for becoming a committer. Candidates for new committers are typically people that are active contributors and community members.
+
+Being an active community member means participating on mailing list discussions, helping to answer questions, being respectful towards others, and following the meritocratic principles of community management. Since the "Apache Way" has a strong focus on the project community, this part is very important.
+
+Of course, contributing code to the project is important as well. A good way to start is contributing improvements, new features, or bugfixes. You need to show that you take responsibility for the code that you contribute, add tests/documentation, and help maintaining it. 
+
+Finally, candidates for new committers are suggested by current committers, mentors, or PMC members, and voted upon by the PMC.
+


[5/5] git commit: Add first files for system internals documentation

Posted by se...@apache.org.
Add first files for system internals documentation

[ci skip]


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

Branch: refs/heads/master
Commit: 5b390d5467cc040b33e6f17329dc87e0693baf5c
Parents: 05ff333
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 27 22:25:36 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Jun 27 22:31:49 2014 +0200

----------------------------------------------------------------------
 docs/internals/general_arch.md           |   8 +
 docs/internals/overview.md               |  39 ++
 docs/internals/projects_dependencies.svg | 567 +++++++++++++++++++++++++
 docs/internals/stack.svg                 | 587 ++++++++++++++++++++++++++
 4 files changed, 1201 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5b390d54/docs/internals/general_arch.md
----------------------------------------------------------------------
diff --git a/docs/internals/general_arch.md b/docs/internals/general_arch.md
new file mode 100644
index 0000000..a2e5ac7
--- /dev/null
+++ b/docs/internals/general_arch.md
@@ -0,0 +1,8 @@
+---
+title:  "General Architecture and Process Model"
+---
+
+<img src="projects_dependencies.svg" alt="The Flink sub-projects and their dependencies" height="400px"/>
+
+
+<img src="stack.svg" alt="The Flink component stack" height="400px"/>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5b390d54/docs/internals/overview.md
----------------------------------------------------------------------
diff --git a/docs/internals/overview.md b/docs/internals/overview.md
new file mode 100644
index 0000000..4c639b4
--- /dev/null
+++ b/docs/internals/overview.md
@@ -0,0 +1,39 @@
+---
+title:  "Overview of Flink System Architecture & Internals"
+---
+
+# Overview
+
+This documentation provides an overview of the architecture of the Flink system
+and its components. It is intended as guide to contributors, and people
+that are interested in the technology behind Flink.
+
+*This documentation is maintained by the contributors of the individual components.
+We kindly ask anyone that adds and changes components to eventually provide a patch
+or pull request that updates these documents as well.*
+
+
+### Architectures and Components
+
+- [General Architecture and Process Model](general_arch.html)
+
+- [Life Cycle of a Program](program_life_cycle.html)
+
+- [Jobs and Scheduling](job_scheduling.html)
+
+- [Distributed Runtime](distributed_runtime.html)
+
+- [Runtime Algorithms and Memory Management](distributed_runtime.html)
+
+- [Program Optimizer](optimizer.html)
+
+- [How-to: Adding a new Operator](add_operator.html)
+
+<!--
+- [Java API, Types, and Type Extraction](types.html)
+-->
+
+<!--
+- [RPC and JobManager Communication](rpc_transfer.html)
+-->
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5b390d54/docs/internals/projects_dependencies.svg
----------------------------------------------------------------------
diff --git a/docs/internals/projects_dependencies.svg b/docs/internals/projects_dependencies.svg
new file mode 100644
index 0000000..a9ba5af
--- /dev/null
+++ b/docs/internals/projects_dependencies.svg
@@ -0,0 +1,567 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<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"
+   version="1.0"
+   width="184.83mm"
+   height="182.98mm"
+   id="svg2985"
+   inkscape:version="0.48.4 r9939"
+   sodipodi:docname="Picture1.emf">
+  <metadata
+     id="metadata3185">
+    <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>
+  <sodipodi:namedview
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1"
+     objecttolerance="10"
+     gridtolerance="10"
+     guidetolerance="10"
+     inkscape:pageopacity="0"
+     inkscape:pageshadow="2"
+     inkscape:window-width="640"
+     inkscape:window-height="480"
+     id="namedview3183"
+     showgrid="false"
+     inkscape:zoom="1.5284853"
+     inkscape:cx="327.45474"
+     inkscape:cy="324.17715"
+     inkscape:window-x="-8"
+     inkscape:window-y="-8"
+     inkscape:window-maximized="1"
+     inkscape:current-layer="svg2985" />
+  <defs
+     id="defs2987" />
+  <g
+     id="g2989">
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 260.48364 21.380326   C 260.48364 18.510862 262.80922 16.185282 265.69744 16.185282   L 378.24423 16.185282   C 381.13245 16.185282 383.45803 18.510862 383.45803 21.380326   L 383.45803 42.216767   C 383.45803 45.086232 381.13245 47.411811 378.24423 47.411811   L 265.69744 47.411811   C 262.80922 47.411811 260.48364 45.086232 260.48364 42.216767   z"
+       id="path2991" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 260.48364 21.380326   C 260.48364 18.510862 262.80922 16.185282 265.69744 16.185282   L 378.24423 16.185282   C 381.13245 16.185282 383.45803 18.510862 383.45803 21.380326   L 383.45803 42.216767   C 383.45803 45.086232 381.13245 47.411811 378.24423 47.411811   L 265.69744 47.411811   C 262.80922 47.411811 260.48364 45.086232 260.48364 42.216767   z"
+       id="path2993" />
+    <text
+       xml:space="preserve"
+       x="275.28831"
+       y="39.668396"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text2995">flink</text>
+    <text
+       xml:space="preserve"
+       x="316.69863"
+       y="39.668396"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text2997">-</text>
+    <text
+       xml:space="preserve"
+       x="325.70087"
+       y="39.668396"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text2999">core</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 130.86073 96.502166   C 130.86073 93.670211 133.1488 91.372763 135.98075 91.372763   L 233.41127 91.372763   C 236.23385 91.372763 238.5313 93.670211 238.5313 96.502166   L 238.5313 116.98227   C 238.5313 119.81422 236.23385 122.10229 233.41127 122.10229   L 135.98075 122.10229   C 133.1488 122.10229 130.86073 119.81422 130.86073 116.98227   z"
+       id="path3001" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5037487px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 130.86073 96.502166   C 130.86073 93.670211 133.1488 91.372763 135.98075 91.372763   L 233.41127 91.372763   C 236.23385 91.372763 238.5313 93.670211 238.5313 96.502166   L 238.5313 116.98227   C 238.5313 119.81422 236.23385 122.10229 233.41127 122.10229   L 135.98075 122.10229   C 133.1488 122.10229 130.86073 119.81422 130.86073 116.98227   z"
+       id="path3003" />
+    <text
+       xml:space="preserve"
+       x="138.4718"
+       y="114.6131"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3005">flink</text>
+    <text
+       xml:space="preserve"
+       x="179.88211"
+       y="114.6131"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3007">-</text>
+    <text
+       xml:space="preserve"
+       x="188.88436"
+       y="114.6131"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3009">java</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 260.48364 96.567807   C 260.48364 93.754607 262.77171 91.466537 265.58491 91.466537   L 363.05295 91.466537   C 365.86615 91.466537 368.15422 93.754607 368.15422 96.567807   L 368.15422 117.0104   C 368.15422 119.8236 365.86615 122.11167 363.05295 122.11167   L 265.58491 122.11167   C 262.77171 122.11167 260.48364 119.8236 260.48364 117.0104   z"
+       id="path3011" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 260.48364 96.567807   C 260.48364 93.754607 262.77171 91.466537 265.58491 91.466537   L 363.05295 91.466537   C 365.86615 91.466537 368.15422 93.754607 368.15422 96.567807   L 368.15422 117.0104   C 368.15422 119.8236 365.86615 122.11167 363.05295 122.11167   L 265.58491 122.11167   C 262.77171 122.11167 260.48364 119.8236 260.48364 117.0104   z"
+       id="path3013" />
+    <text
+       xml:space="preserve"
+       x="264.04551"
+       y="114.65651"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3015">flink</text>
+    <text
+       xml:space="preserve"
+       x="305.45583"
+       y="114.65651"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3017">-</text>
+    <text
+       xml:space="preserve"
+       x="314.45807"
+       y="114.65651"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3019">scala</text>
+    <text
+       xml:space="preserve"
+       x="20.936318"
+       y="28.495681"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3021">c</text>
+    <text
+       xml:space="preserve"
+       x="28.738261"
+       y="28.495681"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3023">ore types, common API, utils,</text>
+    <text
+       xml:space="preserve"
+       x="102.25658"
+       y="46.500168"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3025">core data structures</text>
+    <text
+       xml:space="preserve"
+       x="277.67823"
+       y="145.73346"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3027">Scala API</text>
+    <text
+       xml:space="preserve"
+       x="152.42842"
+       y="145.73346"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3029">Java API</text>
+    <text
+       xml:space="preserve"
+       x="127.97233"
+       y="163.73794"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3031">Old Record API</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 260.48364 198.25564   C 260.48364 195.0861 263.05303 192.51671 266.20382 192.51671   L 399.26822 192.51671   C 402.419 192.51671 404.98839 195.0861 404.98839 198.25564   L 404.98839 221.13634   C 404.98839 224.30588 402.419 226.87527 399.26822 226.87527   L 266.20382 226.87527   C 263.05303 226.87527 260.48364 224.30588 260.48364 221.13634   z"
+       id="path3033" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 260.48364 198.25564   C 260.48364 195.0861 263.05303 192.51671 266.20382 192.51671   L 399.26822 192.51671   C 402.419 192.51671 404.98839 195.0861 404.98839 198.25564   L 404.98839 221.13634   C 404.98839 224.30588 402.419 226.87527 399.26822 226.87527   L 266.20382 226.87527   C 263.05303 226.87527 260.48364 224.30588 260.48364 221.13634   z"
+       id="path3035" />
+    <text
+       xml:space="preserve"
+       x="268.52797"
+       y="217.56563"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3037">flink</text>
+    <text
+       xml:space="preserve"
+       x="309.93829"
+       y="217.56563"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3039">-</text>
+    <text
+       xml:space="preserve"
+       x="318.94053"
+       y="217.56563"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3041">runtime</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 260.48364 262.94051   C 260.48364 259.77097 263.05303 257.22033 266.20382 257.22033   L 399.26822 257.22033   C 402.419 257.22033 404.98839 259.77097 404.98839 262.94051   L 404.98839 285.83996   C 404.98839 288.99074 402.419 291.56013 399.26822 291.56013   L 266.20382 291.56013   C 263.05303 291.56013 260.48364 288.99074 260.48364 285.83996   z"
+       id="path3043" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 260.48364 262.94051   C 260.48364 259.77097 263.05303 257.22033 266.20382 257.22033   L 399.26822 257.22033   C 402.419 257.22033 404.98839 259.77097 404.98839 262.94051   L 404.98839 285.83996   C 404.98839 288.99074 402.419 291.56013 399.26822 291.56013   L 266.20382 291.56013   C 263.05303 291.56013 260.48364 288.99074 260.48364 285.83996   z"
+       id="path3045" />
+    <text
+       xml:space="preserve"
+       x="264.77703"
+       y="282.25536"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3047">flink</text>
+    <text
+       xml:space="preserve"
+       x="306.18735"
+       y="282.25536"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3049">-</text>
+    <text
+       xml:space="preserve"
+       x="315.18959"
+       y="282.25536"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3051">compiler</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 260.48364 326.7439   C 260.48364 323.57436 263.05303 321.02373 266.20382 321.02373   L 399.26822 321.02373   C 402.419 321.02373 404.98839 323.57436 404.98839 326.7439   L 404.98839 349.64335   C 404.98839 352.79414 402.419 355.36353 399.26822 355.36353   L 266.20382 355.36353   C 263.05303 355.36353 260.48364 352.79414 260.48364 349.64335   z"
+       id="path3053" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 260.48364 326.7439   C 260.48364 323.57436 263.05303 321.02373 266.20382 321.02373   L 399.26822 321.02373   C 402.419 321.02373 404.98839 323.57436 404.98839 326.7439   L 404.98839 349.64335   C 404.98839 352.79414 402.419 355.36353 399.26822 355.36353   L 266.20382 355.36353   C 263.05303 355.36353 260.48364 352.79414 260.48364 349.64335   z"
+       id="path3055" />
+    <text
+       xml:space="preserve"
+       x="275.72976"
+       y="346.05788"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3057">flink</text>
+    <text
+       xml:space="preserve"
+       x="317.14008"
+       y="346.05788"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3059">-</text>
+    <text
+       xml:space="preserve"
+       x="326.14232"
+       y="346.05788"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3061">clients</text>
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 333.49558 397.91788   L 333.94569 363.25925   L 331.45132 363.22174   L 331.00121 397.88037   z  M 338.07172 370.17972   L 332.7454 360.74612   L 327.1565 370.02968   C 326.80016 370.62983 326.98771 371.39877 327.58786 371.75511   C 328.16926 372.11145 328.9382 371.92391 329.29454 371.32376   L 329.29454 371.32376   L 333.7769 363.87815   L 331.62011 363.8594   L 335.89618 371.41753   C 336.23376 372.01768 337.00271 372.22398 337.60286 371.8864   C 338.203 371.54881 338.40931 370.77987 338.07172 370.17972   z"
+       id="path3063" />
+    <text
+       xml:space="preserve"
+       x="90.010614"
+       y="280.17295"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3065">compiler </text>
+    <text
+       xml:space="preserve"
+       x="159.62796"
+       y="280.17295"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3067">/ </text>
+    <text
+       xml:space="preserve"
+       x="171.78098"
+       y="280.17295"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3069">optimizer</text>
+    <text
+       xml:space="preserve"
+       x="48.60059"
+       y="208.5999"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3071">l</text>
+    <text
+       xml:space="preserve"
+       x="52.651599"
+       y="208.5999"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3073">ocal/distributed execution,</text>
+    <text
+       xml:space="preserve"
+       x="71.706347"
+       y="226.60439"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3075">runtime </text>
+    <text
+       xml:space="preserve"
+       x="135.92234"
+       y="226.60439"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3077">web monitor</text>
+    <text
+       xml:space="preserve"
+       x="73.125502"
+       y="323.98799"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3079">local/remote execution</text>
+    <text
+       xml:space="preserve"
+       x="124.13821"
+       y="341.99247"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3081">CLI </text>
+    <text
+       xml:space="preserve"
+       x="154.44576"
+       y="341.99247"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3083">client</text>
+    <text
+       xml:space="preserve"
+       x="74.775913"
+       y="359.99696"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3085">compiler web frontend</text>
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 413.72807 12.678158   C 421.54877 12.678158 427.9066 13.72842 427.9066 15.041247   L 427.9066 68.229498   C 427.9066 69.52357 434.24568 70.592586 442.08513 70.592586   C 434.24568 70.592586 427.9066 71.642848 427.9066 72.955675   L 427.9066 126.14393   C 427.9066 127.45675 421.54877 128.50701 413.72807 128.50701 "
+       id="path3087" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 409.35823 181.56398   C 417.17893 181.56398 423.53676 182.61424 423.53676 183.92707   L 423.53676 273.18056   C 423.53676 274.49338 429.87584 275.54364 437.7153 275.54364   C 429.87584 275.54364 423.53676 276.61266 423.53676 277.90673   L 423.53676 367.17897   C 423.53676 368.47305 417.17893 369.54206 409.35823 369.54206 "
+       id="path3089" />
+    <text
+       xml:space="preserve"
+       x="464.00202"
+       y="69.388546"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3091">Needed to </text>
+    <text
+       xml:space="preserve"
+       x="486.95774"
+       y="90.393779"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3093">write </text>
+    <text
+       xml:space="preserve"
+       x="467.00277"
+       y="111.39901"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3095">programs</text>
+    <text
+       xml:space="preserve"
+       x="440.46593"
+       y="265.59638"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3097">Needed </text>
+    <text
+       xml:space="preserve"
+       x="512.93399"
+       y="265.59638"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3099">in addition for </text>
+    <text
+       xml:space="preserve"
+       x="472.87401"
+       y="286.60161"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3101">embedded </text>
+    <text
+       xml:space="preserve"
+       x="571.14849"
+       y="286.60161"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3103">and </text>
+    <text
+       xml:space="preserve"
+       x="445.41717"
+       y="307.60684"
+       style="font-size:17.554373px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3105">distributed execution</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 126.70657 484.20813   C 126.70657 479.61323 130.43875 475.88105 135.05239 475.88105   L 288.46561 475.88105   C 293.07926 475.88105 296.81144 479.61323 296.81144 484.20813   L 296.81144 517.57269   C 296.81144 522.16759 293.07926 525.91852 288.46561 525.91852   L 135.05239 525.91852   C 130.43875 525.91852 126.70657 522.16759 126.70657 517.57269   z"
+       id="path3107" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 126.70657 484.20813   C 126.70657 479.61323 130.43875 475.88105 135.05239 475.88105   L 288.46561 475.88105   C 293.07926 475.88105 296.81144 479.61323 296.81144 484.20813   L 296.81144 517.57269   C 296.81144 522.16759 293.07926 525.91852 288.46561 525.91852   L 135.05239 525.91852   C 130.43875 525.91852 126.70657 522.16759 126.70657 517.57269   z"
+       id="path3109" />
+    <text
+       xml:space="preserve"
+       x="161.02009"
+       y="496.75874"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3111">flink</text>
+    <text
+       xml:space="preserve"
+       x="202.4304"
+       y="496.75874"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3113">-</text>
+    <text
+       xml:space="preserve"
+       x="211.43265"
+       y="496.75874"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3115">java</text>
+    <text
+       xml:space="preserve"
+       x="253.59315"
+       y="496.75874"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3117">-</text>
+    <text
+       xml:space="preserve"
+       x="164.17087"
+       y="520.76472"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3119">examples</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 68.867157 548.64918   C 68.867157 544.12931 72.543072 540.45339 77.062948 540.45339   L 251.57517 540.45339   C 256.09505 540.45339 259.77097 544.12931 259.77097 548.64918   L 259.77097 581.43235   C 259.77097 585.95223 256.09505 589.62814 251.57517 589.62814   L 77.062948 589.62814   C 72.543072 589.62814 68.867157 585.95223 68.867157 581.43235   z"
+       id="path3121" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 68.867157 548.64918   C 68.867157 544.12931 72.543072 540.45339 77.062948 540.45339   L 251.57517 540.45339   C 256.09505 540.45339 259.77097 544.12931 259.77097 548.64918   L 259.77097 581.43235   C 259.77097 585.95223 256.09505 589.62814 251.57517 589.62814   L 77.062948 589.62814   C 72.543072 589.62814 68.867157 585.95223 68.867157 581.43235   z"
+       id="path3123" />
+    <text
+       xml:space="preserve"
+       x="109.65675"
+       y="560.90686"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3125">flink</text>
+    <text
+       xml:space="preserve"
+       x="151.06706"
+       y="560.90686"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3127">-</text>
+    <text
+       xml:space="preserve"
+       x="160.06931"
+       y="560.90686"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3129">scala</text>
+    <text
+       xml:space="preserve"
+       x="210.18179"
+       y="560.90686"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3131">-</text>
+    <text
+       xml:space="preserve"
+       x="116.85854"
+       y="584.91284"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3133">examples</text>
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 273.3681 606.75116   C 273.3681 603.22528 276.25632 600.33706 279.7822 600.33706   L 402.92538 600.33706   C 406.48877 600.33706 409.37699 603.22528 409.37699 606.75116   L 409.37699 632.52008   C 409.37699 636.04595 406.48877 638.93417 402.92538 638.93417   L 279.7822 638.93417   C 276.25632 638.93417 273.3681 636.04595 273.3681 632.52008   z"
+       id="path3135" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.5131261px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 273.3681 606.75116   C 273.3681 603.22528 276.25632 600.33706 279.7822 600.33706   L 402.92538 600.33706   C 406.48877 600.33706 409.37699 603.22528 409.37699 606.75116   L 409.37699 632.52008   C 409.37699 636.04595 406.48877 638.93417 402.92538 638.93417   L 279.7822 638.93417   C 276.25632 638.93417 273.3681 636.04595 273.3681 632.52008   z"
+       id="path3137" />
+    <text
+       xml:space="preserve"
+       x="292.11598"
+       y="627.49492"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3139">flink</text>
+    <text
+       xml:space="preserve"
+       x="333.5263"
+       y="627.49492"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3141">-</text>
+    <text
+       xml:space="preserve"
+       x="342.52854"
+       y="627.49492"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3143">tests</text>
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 298.83695 81.395278   L 308.45809 60.108725   L 306.17002 59.077218   L 296.56763 80.382525   z  M 309.47085 68.098215   L 308.34557 57.333033   L 299.51212 63.615848   C 298.94948 64.009696 298.81819 64.797393 299.2308 65.360033   C 299.62464 65.922673 300.41234 66.053956 300.97498 65.641353   L 308.04549 60.615101   L 306.07625 59.733631   L 306.97648 68.36078   C 307.05149 69.054703 307.6704 69.542325 308.34557 69.467306   C 309.03949 69.411042 309.52711 68.792138 309.47085 68.098215   z"
+       id="path3145" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.0093773361px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 214.32839 88.962788   L 264.78784 60.765138   L 263.56879 58.589596   L 213.10934 86.777869   z  M 260.83998 67.788763   L 266.34448 58.467691   L 255.52303 58.261389   C 254.82911 58.252012 254.25709 58.805275 254.24771 59.48982   C 254.23834 60.183743 254.78222 60.755761 255.47615 60.765138   L 264.15956 60.924553   L 263.09992 59.039708   L 258.69257 66.522823   C 258.33623 67.113595 258.53316 67.882536 259.13331 68.229498   C 259.72408 68.585836 260.49302 68.388912 260.83998 67.788763   z"
+       id="path3147" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 358.60809 184.69601   L 363.55932 174.3997   L 365.97867 169.27967   L 368.34176 164.19716   L 370.64859 159.1709   L 372.86164 154.20092   L 374.96216 149.30595   L 376.95016 144.50475   L 378.78812 139.77857   L 380.47604 135.16492   L 381.97641 130.6638   L 383.30799 126.31272   L 383.87063 124.21219   L 384.39576 122.11167   L 384.88338 120.06741   L 385.27723 118.06066   L 385.63357 116.09142   L 385.93365 114.17844   L 386.13995 112.30298   L 386.30874 110.48377   L 386.38376 108.70208   L 386.40251 106.97665   L 386.365 105.28873   L 386.27123 103.65707   L 386.10244 102.04417   L 385.87738 100.48753   L 385.61482 98.949651   L 385.29599 97.449277   L 384.48954 94.561058   L 383.49554 91.804121   L 382.33275 89.140957   L 381.01992 86.590322   L 379.55706 84.114705   L 377.98166 81.714107   L 376.29374 79.351018   L 374.53081 77.062948   L 372.67409 74.812388   L 370.76112 72.580582   L 368.4918 70.011192   L 370.36727 68.342026   L 372.67409 70.948925   L 374.6
 0582 73.21824   L 376.50005 75.54382   L 378.338 77.906909   L 380.08219 80.326261   L 381.7326 82.839387   L 383.23297 85.446287   L 384.62082 88.14696   L 385.83987 90.96016   L 386.89013 93.885889   L 387.73409 96.942901   L 388.07168 98.518293   L 388.353 100.1312   L 388.59681 101.80036   L 388.7656 103.48828   L 388.87813 105.23247   L 388.91564 107.01416   L 388.89688 108.83336   L 388.78436 110.69008   L 388.63432 112.60305   L 388.39051 114.55354   L 388.09043 116.54153   L 387.73409 118.56704   L 387.30274 120.63005   L 386.83387 122.73058   L 386.28998 124.84985   L 385.68984 127.04415   L 384.35825 131.47025   L 382.82037 136.02764   L 381.11369 140.6788   L 379.25698 145.44248   L 377.26899 150.29994   L 375.14971 155.23242   L 372.9179 160.22117   L 370.61108 165.26617   L 368.22924 170.34869   L 365.80988 175.46871   L 360.8774 185.76503   z  M 369.8984 77.944418   L 367.77912 67.310519   L 378.07544 70.705114   C 378.7131 70.93017 379.06944 71.624093 378.86313 72.280
 507   C 378.63808 72.93692 377.94416 73.293259 377.28774 73.086958   L 369.03569 70.36753   L 370.64859 68.923421   L 372.33651 77.456796   C 372.48655 78.131965 372.03643 78.788378 371.36127 78.919661   C 370.6861 79.050944 370.02968 78.619586 369.8984 77.944418   z"
+       id="path3149" />
+    <path
+       style="fill:#ffffff;fill-rule:evenodd;fill-opacity:1;stroke:none;"
+       d="  M 259.48965 415.00339   C 259.48965 411.89011 262.02153 409.35823 265.1348 409.35823   L 399.36199 409.35823   C 402.47527 409.35823 404.98839 411.89011 404.98839 415.00339   L 404.98839 437.50899   C 404.98839 440.62227 402.47527 443.1354 399.36199 443.1354   L 265.1348 443.1354   C 262.02153 443.1354 259.48965 440.62227 259.48965 437.50899   z"
+       id="path3151" />
+    <path
+       style="fill:none;stroke:#000000;stroke-width:2.4943714px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 259.48965 415.00339   C 259.48965 411.89011 262.02153 409.35823 265.1348 409.35823   L 399.36199 409.35823   C 402.47527 409.35823 404.98839 411.89011 404.98839 415.00339   L 404.98839 437.50899   C 404.98839 440.62227 402.47527 443.1354 399.36199 443.1354   L 265.1348 443.1354   C 262.02153 443.1354 259.48965 440.62227 259.48965 437.50899   z"
+       id="path3153" />
+    <text
+       xml:space="preserve"
+       x="263.06067"
+       y="434.12154"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3155">flink</text>
+    <text
+       xml:space="preserve"
+       x="304.47099"
+       y="434.12154"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3157">-</text>
+    <text
+       xml:space="preserve"
+       x="313.47323"
+       y="434.12154"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3159">test</text>
+    <text
+       xml:space="preserve"
+       x="350.83254"
+       y="434.12154"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3161">-</text>
+    <text
+       xml:space="preserve"
+       x="359.83478"
+       y="434.12154"
+       style="font-size:19.954971px;fill:#000000;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3163">utils</text>
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 333.9832 252.40038   L 333.9832 237.32162   L 331.48883 237.32162   L 331.48883 252.40038   z  M 338.18425 244.18583   L 332.7454 234.84601   L 327.28779 244.18583   C 326.93145 244.78598 327.13775 245.55492 327.7379 245.91126   C 328.31929 246.24885 329.08823 246.04255 329.44457 245.46115   L 333.81441 237.95928   L 331.65762 237.95928   L 336.02746 245.46115   L 336.02746 245.46115   C 336.3838 246.04255 337.15274 246.24885 337.73414 245.91126   C 338.33429 245.55492 338.54059 244.78598 338.18425 244.18583   z"
+       id="path3165" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 333.9832 314.38457   L 333.9832 299.28706   L 331.48883 299.28706   L 331.48883 314.38457   z  M 338.18425 306.17002   L 332.7454 296.81144   L 327.28779 306.17002   C 326.93145 306.77017 327.13775 307.52036 327.7379 307.8767   C 328.31929 308.23304 329.08823 308.02674 329.44457 307.42659   L 333.81441 299.92472   L 331.65762 299.92472   L 336.02746 307.42659   L 336.02746 307.42659   C 336.3838 308.02674 337.15274 308.23304 337.73414 307.8767   C 338.33429 307.52036 338.54059 306.77017 338.18425 306.17002   z"
+       id="path3167" />
+    <text
+       xml:space="preserve"
+       x="165.18987"
+       y="432.03911"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3169">Test </text>
+    <text
+       xml:space="preserve"
+       x="129.63101"
+       y="450.0436"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3171">environments</text>
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 334.03947 589.66565   L 331.05747 457.10763   L 333.55185 457.05136   L 336.53384 589.60939   z  M 327.00647 464.08437   L 332.23902 454.61326   L 337.90293 463.84055   C 338.27802 464.42195 338.09048 465.19089 337.49033 465.54723   C 336.90893 465.92232 336.13999 465.73478 335.78365 465.13463   L 335.78365 465.13463   L 331.22627 457.74529   L 333.40181 457.68902   L 329.18201 465.28466   C 328.84442 465.88481 328.09424 466.10987 327.49409 465.77229   C 326.87518 465.4347 326.66888 464.68451 327.00647 464.08437   z"
+       id="path3173" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.037509345px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 266.72895 617.7789   L 259.33961 617.55385   L 252.1003 617.29128   L 248.49941 617.10374   L 244.97353 616.91619   L 241.56018 616.69113   L 238.18434 616.42857   L 234.92102 616.09098   L 231.77024 615.7534   L 228.73198 615.30329   L 225.80625 614.81567   L 223.03056 614.25303   L 220.44242 613.61537   L 217.9668 612.90269   L 215.71624 612.07748   L 213.61572 611.17726   L 211.70274 610.127   L 209.97731 609.03923   L 208.40192 607.87644   L 206.97656 606.60112   L 205.66374 605.28829   L 204.53846 603.90045   L 203.48819 602.43758   L 202.58797 600.97472   L 201.76276 599.47434   L 201.01258 597.93646   L 200.33741 596.36107   L 199.13711 593.21028   L 198.87454 592.38508   L 201.23763 591.59738   L 201.5002 592.34757   L 202.62548 595.38583   L 203.26314 596.84869   L 203.97582 598.27405   L 204.68849 599.66189   L 205.55121 601.01223   L 206.45143 602.28754   L 207.46418 603.56286   L 208.62697 604.72565   L 209.90229 605.85093   L 211.29014 606.9387   L 212.865
 53 607.95146   L 214.62847 608.85168   L 216.54145 609.7144   L 218.67948 610.50209   L 221.00506 611.17726   L 223.55569 611.81492   L 226.21886 612.34005   L 229.06957 612.82767   L 232.03281 613.24027   L 235.14608 613.61537   L 238.37188 613.91544   L 241.71022 614.17801   L 245.12357 614.40306   L 248.61194 614.62812   L 252.17532 614.77816   L 259.41463 615.04072   L 266.80397 615.26578   z  M 197.11161 600.26204   L 199.24964 589.6469   L 207.42668 596.73616   C 207.95181 597.18627 208.02682 597.97397 207.5392 598.4991   C 207.08909 599.02423 206.30139 599.06174 205.77626 598.61163   L 199.21213 592.94772   L 201.27514 592.23504   L 199.54971 600.74966   C 199.43718 601.42483 198.76202 601.87494 198.08685 601.7249   C 197.41168 601.57487 196.96157 600.93721 197.11161 600.26204   z"
+       id="path3175" />
+    <path
+       style="fill:#000000;fill-rule:nonzero;fill-opacity:1;stroke:#000000;stroke-width:0.018754672px;stroke-linecap:butt;stroke-linejoin:round;stroke-dasharray:none;stroke-opacity:1;"
+       d="  M 310.91496 593.04149   L 306.28255 588.76543   L 301.70641 584.52687   L 297.28031 580.32582   L 295.12352 578.24406   L 293.04175 576.21855   L 291.01625 574.17429   L 289.08452 572.1863   L 287.22781 570.21706   L 285.46487 568.28532   L 283.7957 566.40986   L 282.23906 564.55314   L 280.81371 562.7527   L 279.50088 560.98976   L 278.33809 559.28308   L 277.28783 557.61392   L 276.36885 556.00101   L 275.58115 554.42562   L 274.86848 552.88774   L 274.26833 551.36861   L 273.76195 549.90575   L 273.34935 548.48039   L 272.97425 547.07379   L 272.69293 545.68594   L 272.26158 543.02278   L 271.98026 540.39713   L 271.9615 540.30335   L 274.45587 540.11581   L 274.45587 540.13456   L 274.73719 542.62893   L 275.13104 545.17957   L 275.41236 546.45489   L 275.73119 547.76771   L 276.12504 549.09929   L 276.61266 550.46839   L 277.15655 551.85623   L 277.79421 553.28159   L 278.54439 554.76321   L 279.42586 556.30109   L 280.4011 557.87648   L 281.52639 559.50814   L 282.
 76419 561.19606   L 284.17079 562.959   L 285.67117 564.75945   L 287.30282 566.61616   L 289.02825 568.51038   L 290.86621 570.44211   L 292.79794 572.41135   L 294.78594 574.4181   L 296.86771 576.46236   L 298.98699 578.52538   L 303.41309 582.70767   L 307.97047 586.92747   L 312.62163 591.20353   z  M 268.34185 547.50515   L 273.01176 537.73396   L 279.21956 546.60492   C 279.61341 547.18632 279.46337 547.95526 278.90073 548.34911   C 278.33809 548.74296 277.55039 548.61167 277.15655 548.04903   L 272.18656 540.92226   L 274.34335 540.75347   L 270.59241 548.57416   C 270.29234 549.21182 269.54215 549.47439 268.92325 549.17431   C 268.30434 548.87424 268.04178 548.12405 268.34185 547.50515   z"
+       id="path3177" />
+    <text
+       xml:space="preserve"
+       x="441.23605"
+       y="616.32234"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3179">integration tests</text>
+    <text
+       xml:space="preserve"
+       x="435.68466"
+       y="634.32683"
+       style="font-size:15.003737px;fill:#000000;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;font-family:Verdana;"
+       id="text3181">(entire programs)</text>
+  </g>
+</svg>