You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2015/08/19 18:42:10 UTC

[1/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Repository: flink
Updated Branches:
  refs/heads/master baaa3daca -> bac21bf5d


http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java
deleted file mode 100644
index 05541f5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitionerTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class FieldsPartitionerTest {
-
-	private FieldsPartitioner<Tuple2<String, Integer>> fieldsPartitioner;
-	private StreamRecord<Tuple2<String, Integer>> streamRecord1 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 0));
-	private StreamRecord<Tuple2<String, Integer>> streamRecord2 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 42));
-	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd1 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
-	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd2 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
-
-	@Before
-	public void setPartitioner() {
-		fieldsPartitioner = new FieldsPartitioner<Tuple2<String, Integer>>(new KeySelector<Tuple2<String, Integer>, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String getKey(Tuple2<String, Integer> value) throws Exception {
-				return value.getField(0);
-			}
-		});
-	}
-
-	@Test
-	public void testSelectChannelsLength() {
-		sd1.setInstance(streamRecord1);
-		assertEquals(1, fieldsPartitioner.selectChannels(sd1, 1).length);
-		assertEquals(1, fieldsPartitioner.selectChannels(sd1, 2).length);
-		assertEquals(1, fieldsPartitioner.selectChannels(sd1, 1024).length);
-	}
-
-	@Test
-	public void testSelectChannelsGrouping() {
-		sd1.setInstance(streamRecord1);
-		sd2.setInstance(streamRecord2);
-
-		assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1),
-				fieldsPartitioner.selectChannels(sd2, 1));
-		assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 2),
-				fieldsPartitioner.selectChannels(sd2, 2));
-		assertArrayEquals(fieldsPartitioner.selectChannels(sd1, 1024),
-				fieldsPartitioner.selectChannels(sd2, 1024));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
index 856f7aa..f7bd739 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
@@ -27,14 +27,14 @@ import org.junit.Test;
 
 public class ForwardPartitionerTest {
 
-	private RebalancePartitioner<Tuple> forwardPartitioner;
+	private ForwardPartitioner<Tuple> forwardPartitioner;
 	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
 	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
 			null);
 
 	@Before
 	public void setPartitioner() {
-		forwardPartitioner = new RebalancePartitioner<Tuple>(true);
+		forwardPartitioner = new ForwardPartitioner<Tuple>();
 	}
 
 	@Test
@@ -49,7 +49,7 @@ public class ForwardPartitionerTest {
 	public void testSelectChannelsInterval() {
 		sd.setInstance(streamRecord);
 		assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]);
-		assertEquals(1, forwardPartitioner.selectChannels(sd, 2)[0]);
-		assertEquals(2, forwardPartitioner.selectChannels(sd, 1024)[0]);
+		assertEquals(0, forwardPartitioner.selectChannels(sd, 2)[0]);
+		assertEquals(0, forwardPartitioner.selectChannels(sd, 1024)[0]);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
new file mode 100644
index 0000000..6dbf932
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.partitioner;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Before;
+import org.junit.Test;
+
+public class HashPartitionerTest {
+
+	private HashPartitioner<Tuple2<String, Integer>> hashPartitioner;
+	private StreamRecord<Tuple2<String, Integer>> streamRecord1 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 0));
+	private StreamRecord<Tuple2<String, Integer>> streamRecord2 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 42));
+	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd1 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
+	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd2 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
+
+	@Before
+	public void setPartitioner() {
+		hashPartitioner = new HashPartitioner<Tuple2<String, Integer>>(new KeySelector<Tuple2<String, Integer>, String>() {
+
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public String getKey(Tuple2<String, Integer> value) throws Exception {
+				return value.getField(0);
+			}
+		});
+	}
+
+	@Test
+	public void testSelectChannelsLength() {
+		sd1.setInstance(streamRecord1);
+		assertEquals(1, hashPartitioner.selectChannels(sd1, 1).length);
+		assertEquals(1, hashPartitioner.selectChannels(sd1, 2).length);
+		assertEquals(1, hashPartitioner.selectChannels(sd1, 1024).length);
+	}
+
+	@Test
+	public void testSelectChannelsGrouping() {
+		sd1.setInstance(streamRecord1);
+		sd2.setInstance(streamRecord2);
+
+		assertArrayEquals(hashPartitioner.selectChannels(sd1, 1),
+				hashPartitioner.selectChannels(sd2, 1));
+		assertArrayEquals(hashPartitioner.selectChannels(sd1, 2),
+				hashPartitioner.selectChannels(sd2, 2));
+		assertArrayEquals(hashPartitioner.selectChannels(sd1, 1024),
+				hashPartitioner.selectChannels(sd2, 1024));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
new file mode 100644
index 0000000..aa70e8a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.partitioner;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RebalancePartitionerTest {
+	
+	private RebalancePartitioner<Tuple> distributePartitioner;
+	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
+	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
+			null);
+	
+	@Before
+	public void setPartitioner() {
+		distributePartitioner = new RebalancePartitioner<Tuple>();
+	}
+	
+	@Test
+	public void testSelectChannelsLength() {
+		sd.setInstance(streamRecord);
+		assertEquals(1, distributePartitioner.selectChannels(sd, 1).length);
+		assertEquals(1, distributePartitioner.selectChannels(sd, 2).length);
+		assertEquals(1, distributePartitioner.selectChannels(sd, 1024).length);
+	}
+	
+	@Test
+	public void testSelectChannelsInterval() {
+		sd.setInstance(streamRecord);
+		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
+		assertEquals(1, distributePartitioner.selectChannels(sd, 3)[0]);
+		assertEquals(2, distributePartitioner.selectChannels(sd, 3)[0]);
+		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
index 52de8aa..a41cb8c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
@@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.NoOpSink;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -111,7 +112,8 @@ public class TimestampITCase {
 		source1
 				.map(new IdentityMap())
 				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator());
+				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator())
+				.addSink(new NoOpSink<Integer>());
 
 		env.execute();
 
@@ -149,7 +151,9 @@ public class TimestampITCase {
 		source1
 				.map(new IdentityMap())
 				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator());
+				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator())
+				.addSink(new NoOpSink<Integer>());
+
 
 		env.execute();
 	}
@@ -176,7 +180,9 @@ public class TimestampITCase {
 		source1
 				.map(new IdentityMap())
 				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator());
+				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator())
+				.addSink(new NoOpSink<Integer>());
+
 
 		env.execute();
 	}
@@ -194,7 +200,8 @@ public class TimestampITCase {
 		DataStream<Integer> source1 = env.addSource(new MyErroneousTimestampSource());
 
 		source1
-				.map(new IdentityMap());
+				.map(new IdentityMap())
+				.addSink(new NoOpSink<Integer>());
 
 		env.execute();
 	}
@@ -212,7 +219,8 @@ public class TimestampITCase {
 		DataStream<Integer> source1 = env.addSource(new MyErroneousSource());
 
 		source1
-				.map(new IdentityMap());
+				.map(new IdentityMap())
+				.addSink(new NoOpSink<Integer>());
 
 		env.execute();
 	}
@@ -230,7 +238,8 @@ public class TimestampITCase {
 		DataStream<Integer> source1 = env.addSource(new MyErroneousWatermarkSource());
 
 		source1
-				.map(new IdentityMap());
+				.map(new IdentityMap())
+				.addSink(new NoOpSink<Integer>());
 
 		env.execute();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
new file mode 100644
index 0000000..9820ef8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+
+import java.util.Arrays;
+
+public class EvenOddOutputSelector implements OutputSelector<Integer> {
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public Iterable<String> select(Integer value) {
+		return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
new file mode 100644
index 0000000..bcb5691
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.api.common.functions.MapFunction;
+
+public class NoOpIntMap implements MapFunction<Integer, Integer> {
+	private static final long serialVersionUID = 1L;
+
+	public Integer map(Integer value) throws Exception {
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
new file mode 100644
index 0000000..d398121
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+
+public final class NoOpSink<T> extends RichSinkFunction<T> {
+	public void invoke(T tuple) {
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
new file mode 100644
index 0000000..a46ff55
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+public final class ReceiveCheckNoOpSink<T> extends RichSinkFunction<T> {
+	private List<T> received;
+
+	public void invoke(T tuple) {
+		received.add(tuple);
+	}
+
+	public void open(Configuration conf) {
+		received = new ArrayList<T>();
+	}
+
+	public void close() {
+		assertTrue(received.size() > 0);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index 1a6ef0f..fbcda1c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.util;
 
+import com.google.common.base.Preconditions;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -44,7 +45,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	}
 
 	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism){
-		this.executor = executor;
+		this.executor = Preconditions.checkNotNull(executor);
 		setDefaultLocalParallelism(parallelism);
 		setParallelism(parallelism);
 	}
@@ -56,7 +57,8 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		return execute(streamGraph.getJobGraph(jobName));
+		JobExecutionResult result = execute(getStreamGraph().getJobGraph(jobName));
+		return result;
 	}
 	
 	public JobExecutionResult execute(JobGraph jobGraph) throws Exception {
@@ -81,6 +83,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 				throw e;
 			}
 		} finally {
+			transformations.clear();
 			if (internalExecutor){
 				executor.shutdown();
 			}
@@ -161,7 +164,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 			}
 		};
 
-		initializeFromFactory(factory);
+		initializeContextEnvironment(factory);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 5e62331..2bb6a6a 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -82,10 +82,6 @@ class DataStream[T](javaStream: JavaStream[T]) {
    */
   def getParallelism = javaStream.getParallelism
   
-  def getPartitioner = javaStream.getPartitioner
-  
-  def getSelectedNames = javaStream.getSelectedNames
-
   /**
    * Returns the execution config.
    */
@@ -403,7 +399,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
     val iterativeStream = javaStream.iterate(maxWaitTimeMillis)
 
     val (feedback, output) = stepFunction(new DataStream[T](iterativeStream))
-    iterativeStream.closeWith(feedback.getJavaStream, keepPartitioning)
+    iterativeStream.closeWith(feedback.getJavaStream)
     output
   }
   
@@ -703,7 +699,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * written.
    *
    */
-  def print(): DataStream[T] = javaStream.print()
+  def print(): DataStreamSink[T] = javaStream.print()
 
   /**
    * Writes a DataStream to the standard output stream (stderr).
@@ -722,7 +718,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * is written.
    *
    */
-  def writeAsText(path: String, millis: Long = 0): DataStream[T] =
+  def writeAsText(path: String, millis: Long = 0): DataStreamSink[T] =
     javaStream.writeAsText(path, millis)
 
   /**
@@ -737,7 +733,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
       millis: Long = 0,
       rowDelimiter: String = ScalaCsvOutputFormat.DEFAULT_LINE_DELIMITER,
       fieldDelimiter: String = ScalaCsvOutputFormat.DEFAULT_FIELD_DELIMITER,
-      writeMode: FileSystem.WriteMode = null): DataStream[T] = {
+      writeMode: FileSystem.WriteMode = null): DataStreamSink[T] = {
     require(javaStream.getType.isTupleType, "CSV output can only be used with Tuple DataSets.")
     val of = new ScalaCsvOutputFormat[Product](new Path(path), rowDelimiter, fieldDelimiter)
     if (writeMode != null) {
@@ -758,8 +754,12 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * Writes the DataStream to a socket as a byte array. The format of the output is
    * specified by a [[SerializationSchema]].
    */
-  def writeToSocket(hostname: String, port: Integer, schema: SerializationSchema[T, Array[Byte]]):
-    DataStream[T] = javaStream.writeToSocket(hostname, port, schema)
+  def writeToSocket(
+      hostname: String,
+      port: Integer,
+      schema: SerializationSchema[T, Array[Byte]]): DataStreamSink[T] = {
+    javaStream.writeToSocket(hostname, port, schema)
+  }
 
   /**
    * Adds the given sink to this DataStream. Only streams with sinks added
@@ -767,7 +767,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * method is called.
    *
    */
-  def addSink(sinkFunction: SinkFunction[T]): DataStream[T] =
+  def addSink(sinkFunction: SinkFunction[T]): DataStreamSink[T] =
     javaStream.addSink(sinkFunction)
 
   /**
@@ -776,7 +776,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * method is called.
    *
    */
-  def addSink(fun: T => Unit): DataStream[T] = {
+  def addSink(fun: T => Unit): DataStreamSink[T] = {
     if (fun == null) {
       throw new NullPointerException("Sink function must not be null.")
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
index 7033218..0e01eee 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
@@ -82,18 +82,14 @@ object StreamCrossOperator {
     def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = {
 
       val cleanCrossWindowFunction = clean(getCrossWindowFunction(op, fun))
-      val operator = new CoStreamWindow[I1, I2, R](
+
+      op.input1.connect(op.input2).addGeneralWindowCombine(
         cleanCrossWindowFunction,
+        implicitly[TypeInformation[R]],
         op.windowSize,
         op.slideInterval,
         op.timeStamp1,
         op.timeStamp2)
-
-      javaStream.getExecutionEnvironment().getStreamGraph().setOperator(javaStream.getId(),
-        operator)
-        
-      val js = javaStream.asInstanceOf[SingleOutputStreamOperator[R,_]]
-      js.returns(implicitly[TypeInformation[R]]).asInstanceOf[SingleOutputStreamOperator[R,_]]
     }
     
     override def every(length: Long, timeUnit: TimeUnit): CrossWindow[I1, I2] = {

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
index 09329ca..e872851 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
@@ -182,8 +182,9 @@ object StreamJoinOperator {
     }
   }
 
-  class JoinedStream[I1, I2](jp: JoinPredicate[I1, I2], javaStream: JavaStream[(I1, I2)]) extends 
-  DataStream[(I1, I2)](javaStream) {
+  class JoinedStream[I1, I2](
+      jp: JoinPredicate[I1, I2],
+      javaStream: JavaStream[(I1, I2)]) extends DataStream[(I1, I2)](javaStream) {
 
     private val op = jp.op
 
@@ -194,18 +195,15 @@ object StreamJoinOperator {
     def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = {
 
       val cleanFun = clean(getJoinWindowFunction(jp, fun))
-      val operator = new CoStreamWindow[I1, I2, R](
-        cleanFun,
-        op.windowSize,
-        op.slideInterval,
-        op.timeStamp1,
-        op.timeStamp2)
-
-      javaStream.getExecutionEnvironment().getStreamGraph().setOperator(javaStream.getId(),
-        operator)
-
-      val js = javaStream.asInstanceOf[SingleOutputStreamOperator[R,_]]
-      js.returns(implicitly[TypeInformation[R]]).asInstanceOf[SingleOutputStreamOperator[R,_]]
+
+      op.input1.groupBy(jp.keys1).connect(op.input2.groupBy(jp.keys2))
+        .addGeneralWindowCombine[R](
+          cleanFun,
+          implicitly[TypeInformation[R]],
+          op.windowSize,
+          op.slideInterval,
+          op.timeStamp1,
+          op.timeStamp2)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index 39a8fe6..8b4d527 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -28,21 +28,21 @@ import org.apache.flink.streaming.api.graph.{StreamEdge, StreamGraph}
 import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, StreamOperator}
 import org.apache.flink.streaming.api.windowing.helper.Count
 import org.apache.flink.streaming.runtime.partitioner._
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.util.Collector
 import org.junit.Assert.fail
 import org.junit.Test
 import org.apache.flink.streaming.api.scala.function.StatefulFunction
 
-class DataStreamTest {
-
-  private val parallelism = 2
+class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
   @Test
   def testNaming(): Unit = {
-    val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism)
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
 
-    val source1 = env.generateSequence(0, 0).name("testSource1")
-    assert("testSource1" == source1.getName)
+    val source1Operator = env.generateSequence(0, 0).name("testSource1")
+    val source1 = source1Operator
+    assert("testSource1" == source1Operator.getName)
 
     val dataStream1 = source1
       .map(x => 0L)
@@ -70,7 +70,7 @@ class DataStreamTest {
     windowed.name("testWindowFold")
     assert("testWindowFold" == windowed.getName)
 
-    windowed.flatten()
+    windowed.flatten().print()
 
     val plan = env.getExecutionPlan
 
@@ -88,8 +88,7 @@ class DataStreamTest {
    */
   @Test
   def testPartitioning(): Unit = {
-    val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism);
-    val graph: StreamGraph = env.getStreamGraph;
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
 
     val src1: DataStream[(Long, Long)] = env.fromElements((0L, 0L))
     val src2: DataStream[(Long, Long)] = env.fromElements((0L, 0L))
@@ -101,10 +100,14 @@ class DataStreamTest {
     val group3 = src1.groupBy("_1")
     val group4 = src1.groupBy(x => x._1)
 
-    assert(isPartitioned(graph.getStreamEdge(group1.getId, createDownStreamId(group1))))
-    assert(isPartitioned(graph.getStreamEdge(group2.getId, createDownStreamId(group2))))
-    assert(isPartitioned(graph.getStreamEdge(group3.getId, createDownStreamId(group3))))
-    assert(isPartitioned(graph.getStreamEdge(group4.getId, createDownStreamId(group4))))
+    val gid1 = createDownStreamId(group1)
+    val gid2 = createDownStreamId(group2)
+    val gid3 = createDownStreamId(group3)
+    val gid4 = createDownStreamId(group4)
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid1)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid2)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid3)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid4)))
 
     //Testing DataStream partitioning
     val partition1: DataStream[_] = src1.partitionByHash(0)
@@ -112,10 +115,15 @@ class DataStreamTest {
     val partition3: DataStream[_] = src1.partitionByHash("_1")
     val partition4: DataStream[_] = src1.partitionByHash((x : (Long, Long)) => x._1);
 
-    assert(isPartitioned(graph.getStreamEdge(partition1.getId, createDownStreamId(partition1))))
-    assert(isPartitioned(graph.getStreamEdge(partition2.getId, createDownStreamId(partition2))))
-    assert(isPartitioned(graph.getStreamEdge(partition3.getId, createDownStreamId(partition3))))
-    assert(isPartitioned(graph.getStreamEdge(partition4.getId, createDownStreamId(partition4))))
+    val pid1 = createDownStreamId(partition1)
+    val pid2 = createDownStreamId(partition2)
+    val pid3 = createDownStreamId(partition3)
+    val pid4 = createDownStreamId(partition4)
+
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid1)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid2)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid3)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid4)))
 
     // Testing DataStream custom partitioning
     val longPartitioner: Partitioner[Long] = new Partitioner[Long] {
@@ -129,12 +137,12 @@ class DataStreamTest {
     val customPartition4: DataStream[_] =
       src1.partitionCustom(longPartitioner, (x : (Long, Long)) => x._1)
 
-    assert(isCustomPartitioned(
-      graph.getStreamEdge(customPartition1.getId, createDownStreamId(customPartition1))))
-    assert(isCustomPartitioned(
-      graph.getStreamEdge(customPartition3.getId, createDownStreamId(customPartition3))))
-    assert(isCustomPartitioned(
-      graph.getStreamEdge(customPartition4.getId, createDownStreamId(customPartition4))))
+    val cpid1 = createDownStreamId(customPartition1)
+    val cpid2 = createDownStreamId(customPartition3)
+    val cpid3 = createDownStreamId(customPartition4)
+    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid1)))
+    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid2)))
+    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid3)))
 
     //Testing ConnectedDataStream grouping
     val connectedGroup1: ConnectedDataStream[_, _] = connected.groupBy(0, 0)
@@ -153,20 +161,20 @@ class DataStreamTest {
     val connectedGroup5: ConnectedDataStream[_, _] = connected.groupBy(x => x._1, x => x._1)
     val downStreamId5: Integer = createDownStreamId(connectedGroup5)
 
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup1.getFirst.getId, downStreamId1)))
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup1.getSecond.getId, downStreamId1)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId1)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId1)))
 
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup2.getFirst.getId, downStreamId2)))
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup2.getSecond.getId, downStreamId2)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId2)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId2)))
 
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup3.getFirst.getId, downStreamId3)))
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup3.getSecond.getId, downStreamId3)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId3)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId3)))
 
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup4.getFirst.getId, downStreamId4)))
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup4.getSecond.getId, downStreamId4)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId4)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId4)))
 
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup5.getFirst.getId, downStreamId5)))
-    assert(isPartitioned(graph.getStreamEdge(connectedGroup5.getSecond.getId, downStreamId5)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId5)))
+    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId5)))
 
     //Testing ConnectedDataStream partitioning
     val connectedPartition1: ConnectedDataStream[_, _] = connected.partitionByHash(0, 0)
@@ -188,38 +196,38 @@ class DataStreamTest {
     val connectDownStreamId5: Integer = createDownStreamId(connectedPartition5)
 
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition1.getFirst.getId, connectDownStreamId1))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId1))
     )
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition1.getSecond.getId, connectDownStreamId1))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId1))
     )
 
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition2.getFirst.getId, connectDownStreamId2))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId2))
     )
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition2.getSecond.getId, connectDownStreamId2))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId2))
     )
 
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition3.getFirst.getId, connectDownStreamId3))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId3))
     )
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition3.getSecond.getId, connectDownStreamId3))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId3))
     )
 
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition4.getFirst.getId, connectDownStreamId4))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId4))
     )
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition4.getSecond.getId, connectDownStreamId4))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId4))
     )
 
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition5.getFirst.getId, connectDownStreamId5))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId5))
     )
     assert(
-      isPartitioned(graph.getStreamEdge(connectedPartition5.getSecond.getId, connectDownStreamId5))
+      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId5))
     )
   }
 
@@ -230,20 +238,19 @@ class DataStreamTest {
   def testParallelism {
     val env: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment(10)
 
-    val graph: StreamGraph = env.getStreamGraph
-
     val src = env.fromElements(new Tuple2[Long, Long](0L, 0L))
     val map = src.map(x => 0L)
     val windowed: DataStream[Long] = map
       .window(Count.of(10))
       .foldWindow(0L, (x: Long, y: Long) => 0L)
       .flatten
+    windowed.print()
     val sink = map.addSink(x => {})
 
-    assert(1 == graph.getStreamNode(src.getId).getParallelism)
-    assert(10 == graph.getStreamNode(map.getId).getParallelism)
-    assert(10 == graph.getStreamNode(windowed.getId).getParallelism)
-    assert(10 == graph.getStreamNode(sink.getId).getParallelism)
+    assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism)
+    assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
+    assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism)
+    assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
 
     try {
       src.setParallelism(3)
@@ -255,28 +262,31 @@ class DataStreamTest {
     }
 
     env.setParallelism(7)
-    assert(1 == graph.getStreamNode(src.getId).getParallelism)
-    assert(7 == graph.getStreamNode(map.getId).getParallelism)
-    assert(7 == graph.getStreamNode(windowed.getId).getParallelism)
-    assert(7 == graph.getStreamNode(sink.getId).getParallelism)
+    // the parallelism does not change since some windowing code takes the parallelism from
+    // input operations and that cannot change dynamically
+    assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism)
+    assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
+    assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism)
+    assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
 
     val parallelSource = env.generateSequence(0, 0)
+    parallelSource.print()
 
-    assert(7 == graph.getStreamNode(parallelSource.getId).getParallelism)
+    assert(7 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism)
 
     parallelSource.setParallelism(3)
-    assert(3 == graph.getStreamNode(parallelSource.getId).getParallelism)
+    assert(3 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism)
 
     map.setParallelism(2)
-    assert(2 == graph.getStreamNode(map.getId).getParallelism)
+    assert(2 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
 
     sink.setParallelism(4)
-    assert(4 == graph.getStreamNode(sink.getId).getParallelism)
+    assert(4 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
   }
 
   @Test
   def testTypeInfo {
-    val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism)
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
 
     val src1: DataStream[Long] = env.generateSequence(0, 0)
     assert(TypeExtractor.getForClass(classOf[Long]) == src1.getType)
@@ -299,15 +309,14 @@ class DataStreamTest {
   }
 
   @Test def operatorTest {
-    val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism)
-
-    val streamGraph = env.getStreamGraph
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
 
     val src = env.generateSequence(0, 0)
 
     val mapFunction = new MapFunction[Long, Int] {
       override def map(value: Long): Int = 0
-    };
+    }
+
     val map = src.map(mapFunction)
     assert(mapFunction == getFunctionForDataStream(map))
     assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[_, _]])
@@ -364,7 +373,7 @@ class DataStreamTest {
         asInstanceOf[StatefulFunction[_, _, _]].partitioned)
    
     try {
-      streamGraph.getStreamEdge(map.getId, unionFilter.getId)
+      env.getStreamGraph.getStreamEdge(map.getId, unionFilter.getId)
     }
     catch {
       case e: Throwable => {
@@ -373,7 +382,7 @@ class DataStreamTest {
     }
 
     try {
-      streamGraph.getStreamEdge(flatMap.getId, unionFilter.getId)
+      env.getStreamGraph.getStreamEdge(flatMap.getId, unionFilter.getId)
     }
     catch {
       case e: Throwable => {
@@ -386,17 +395,19 @@ class DataStreamTest {
     }
 
     val split = unionFilter.split(outputSelector)
-    val outputSelectors = streamGraph.getStreamNode(split.getId).getOutputSelectors
+    split.print()
+    val outputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors
     assert(1 == outputSelectors.size)
     assert(outputSelector == outputSelectors.get(0))
 
-    unionFilter.split(x => List("a"))
-    val moreOutputSelectors = streamGraph.getStreamNode(split.getId).getOutputSelectors
+    unionFilter.split(x => List("a")).print()
+    val moreOutputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors
     assert(2 == moreOutputSelectors.size)
 
     val select = split.select("a")
     val sink = select.print
-    val splitEdge = streamGraph.getStreamEdge(select.getId, sink.getId)
+    val splitEdge =
+      env.getStreamGraph.getStreamEdge(unionFilter.getId, sink.getTransformation.getId)
     assert("a" == splitEdge.getSelectedNames.get(0))
 
     val foldFunction = new FoldFunction[Int, String] {
@@ -421,7 +432,7 @@ class DataStreamTest {
     assert(coMapFunction == getFunctionForDataStream(coMap))
 
     try {
-      streamGraph.getStreamEdge(fold.getId, coMap.getId)
+      env.getStreamGraph.getStreamEdge(fold.getId, coMap.getId)
     }
     catch {
       case e: Throwable => {
@@ -429,7 +440,7 @@ class DataStreamTest {
       }
     }
     try {
-      streamGraph.getStreamEdge(flatMap.getId, coMap.getId)
+      env.getStreamGraph.getStreamEdge(flatMap.getId, coMap.getId)
     }
     catch {
       case e: Throwable => {
@@ -440,51 +451,51 @@ class DataStreamTest {
 
   @Test
   def testChannelSelectors {
-    val env = StreamExecutionEnvironment.createLocalEnvironment(parallelism)
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
 
-    val streamGraph = env.getStreamGraph
     val src = env.generateSequence(0, 0)
 
     val broadcast = src.broadcast
     val broadcastSink = broadcast.print
-    val broadcastPartitioner = streamGraph
-      .getStreamEdge(broadcast.getId, broadcastSink.getId).getPartitioner
+    val broadcastPartitioner = env.getStreamGraph
+      .getStreamEdge(src.getId, broadcastSink.getTransformation.getId).getPartitioner
     assert(broadcastPartitioner.isInstanceOf[BroadcastPartitioner[_]])
 
     val shuffle: DataStream[Long] = src.shuffle
     val shuffleSink = shuffle.print
-    val shufflePartitioner = streamGraph
-      .getStreamEdge(shuffle.getId, shuffleSink.getId).getPartitioner
+    val shufflePartitioner = env.getStreamGraph
+      .getStreamEdge(src.getId, shuffleSink.getTransformation.getId).getPartitioner
     assert(shufflePartitioner.isInstanceOf[ShufflePartitioner[_]])
 
     val forward: DataStream[Long] = src.forward
     val forwardSink = forward.print
-    val forwardPartitioner = streamGraph
-      .getStreamEdge(forward.getId, forwardSink.getId).getPartitioner
-    assert(forwardPartitioner.isInstanceOf[RebalancePartitioner[_]])
+    val forwardPartitioner = env.getStreamGraph
+      .getStreamEdge(src.getId, forwardSink.getTransformation.getId).getPartitioner
+    assert(forwardPartitioner.isInstanceOf[ForwardPartitioner[_]])
 
     val rebalance: DataStream[Long] = src.rebalance
     val rebalanceSink = rebalance.print
-    val rebalancePartitioner = streamGraph
-      .getStreamEdge(rebalance.getId, rebalanceSink.getId).getPartitioner
+    val rebalancePartitioner = env.getStreamGraph
+      .getStreamEdge(src.getId, rebalanceSink.getTransformation.getId).getPartitioner
     assert(rebalancePartitioner.isInstanceOf[RebalancePartitioner[_]])
 
     val global: DataStream[Long] = src.global
     val globalSink = global.print
-    val globalPartitioner = streamGraph
-      .getStreamEdge(global.getId, globalSink.getId).getPartitioner
+    val globalPartitioner = env.getStreamGraph
+      .getStreamEdge(src.getId, globalSink.getTransformation.getId).getPartitioner
     assert(globalPartitioner.isInstanceOf[GlobalPartitioner[_]])
   }
 
   @Test
   def testIterations {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val source = env.fromElements(1, 2, 3)
+    // we need to rebalance before iteration
+    val source = env.fromElements(1, 2, 3).map { t: Int => t }
 
     val iterated = source.iterate((input: ConnectedDataStream[Int, String]) => {
       val head = input.map(i => (i + 1).toString, s => s)
       (head.filter(_ == "2"), head.filter(_ != "2"))
-    }, 1000)
+    }, 1000).print()
 
     val iterated2 = source.iterate((input: DataStream[Int]) => 
       (input.map(_ + 1), input.map(_.toString)), 2000)
@@ -493,7 +504,7 @@ class DataStreamTest {
       val invalid = source.iterate((input: ConnectedDataStream[Int, String]) => {
         val head = input.partitionByHash(1, 1).map(i => (i + 1).toString, s => s)
         (head.filter(_ == "2"), head.filter(_ != "2"))
-      }, 1000)
+      }, 1000).print()
       fail
     } catch {
       case uoe: UnsupportedOperationException =>
@@ -502,7 +513,7 @@ class DataStreamTest {
 
     val sg = env.getStreamGraph
 
-    assert(sg.getStreamLoops().size() == 2)
+    assert(sg.getIterationSourceSinkPairs().size() == 2)
   }
 
   /////////////////////////////////////////////////////////////
@@ -510,19 +521,21 @@ class DataStreamTest {
   /////////////////////////////////////////////////////////////
 
   private def getFunctionForDataStream(dataStream: DataStream[_]): Function = {
+    dataStream.print()
     val operator = getOperatorForDataStream(dataStream)
       .asInstanceOf[AbstractUdfStreamOperator[_, _]]
     return operator.getUserFunction.asInstanceOf[Function]
   }
 
   private def getOperatorForDataStream(dataStream: DataStream[_]): StreamOperator[_] = {
+    dataStream.print()
     val env = dataStream.getJavaStream.getExecutionEnvironment
     val streamGraph: StreamGraph = env.getStreamGraph
     streamGraph.getStreamNode(dataStream.getId).getOperator
   }
 
   private def isPartitioned(edge: StreamEdge): Boolean = {
-    return edge.getPartitioner.isInstanceOf[FieldsPartitioner[_]]
+    return edge.getPartitioner.isInstanceOf[HashPartitioner[_]]
   }
 
   private def isCustomPartitioned(edge: StreamEdge): Boolean = {
@@ -530,11 +543,13 @@ class DataStreamTest {
   }
 
   private def createDownStreamId(dataStream: DataStream[_]): Integer = {
-    return dataStream.print.getId
+    return dataStream.print.getTransformation.getId
   }
 
   private def createDownStreamId(dataStream: ConnectedDataStream[_, _]): Integer = {
-    return dataStream.map(x => 0, x => 0).getId
+    val m = dataStream.map(x => 0, x => 0)
+    m.print()
+    m.getId
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
index 88b0f4f..0d573a9 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
@@ -18,14 +18,16 @@
 package org.apache.flink.streaming.api.scala
 
 import org.apache.flink.streaming.util.SocketOutputTestBase.DummyStringSchema
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema
+import org.apache.flink.test.util.MultipleProgramsTestBase
 
 import scala.language.existentials
 
 /**
  * Test programs for built in output formats. Invoked from {@link OutputFormatTest}.
  */
-object OutputFormatTestPrograms  {
+object OutputFormatTestPrograms {
 
   def wordCountToText(input : String, outputPath : String) : Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
index 5fa007c..650fd7e 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
@@ -29,13 +29,13 @@ object StateTestPrograms {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     
     // test stateful map
-    env.generateSequence(0, 10).setParallelism(1).
-      mapWithState((in, count: Option[Long]) =>
+    env.generateSequence(0, 10).setParallelism(1)
+      .mapWithState((in, count: Option[Long]) =>
         count match {
           case Some(c) => ((in - c), Some(c + 1))
           case None => (in, Some(1L))
-        }).setParallelism(1).
-      addSink(new RichSinkFunction[Long]() {
+        }).setParallelism(1)
+      .addSink(new RichSinkFunction[Long]() {
         var allZero = true
         override def invoke(in: Long) = {
           if (in != 0) allZero = false
@@ -50,8 +50,8 @@ object StateTestPrograms {
       s match {
         case Some(s) => (w.split(" ").toList.map(s + _), Some(w))
         case None => (List(w), Some(w))
-      }).setParallelism(1).
-      addSink(new RichSinkFunction[String]() {
+      }).setParallelism(1)
+      .addSink(new RichSinkFunction[String]() {
         val received = new HashSet[String]()
         override def invoke(in: String) = { received.add(in) }
         override def close() = {

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index 7ebc161..e7d9953 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -41,6 +41,7 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "org.apache.flink.streaming.api.datastream.DataStream.getType",
       "org.apache.flink.streaming.api.datastream.DataStream.copy",
       "org.apache.flink.streaming.api.datastream.DataStream.transform",
+      "org.apache.flink.streaming.api.datastream.DataStream.getTransformation",
       "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.copy",
       "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getExecutionEnvironment",
       "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getType1",
@@ -50,6 +51,16 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getType",
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig",
 
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.isChainingEnabled",
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." +
+        "getStateHandleProvider",
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointInterval",
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.addOperator",
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointingMode",
+      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." +
+        "isForceCheckpointing",
+
+
       // TypeHints are only needed for Java API, Scala API doesn't need them
       "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.returns"
     )

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
index 020919e..ae31d95 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
@@ -24,7 +24,6 @@ import akka.actor.Status;
 import akka.pattern.Patterns;
 import akka.testkit.JavaTestKit;
 import akka.util.Timeout;
-import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.Plan;
@@ -163,7 +162,7 @@ public class AccumulatorLiveITCase {
 	@Test
 	public void testStreaming() throws Exception {
 
-		StreamExecutionEnvironment env = new StreamJobExtractor();
+		StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment();
 		env.setParallelism(1);
 
 		DataStream<String> input = env.fromCollection(inputData);
@@ -171,9 +170,8 @@ public class AccumulatorLiveITCase {
 				.flatMap(new NotifyingMapper())
 				.write(new NotifyingOutputFormat(), 1000).disableChaining();
 
-		env.execute();
 
-		jobGraph = ((StreamJobExtractor) env).graph;
+		jobGraph = env.getStreamGraph().getJobGraph();
 		jobID = jobGraph.getJobID();
 
 		verifyResults();
@@ -346,6 +344,7 @@ public class AccumulatorLiveITCase {
 	 * UDF that notifies when it changes the accumulator values
 	 */
 	private static class NotifyingMapper extends RichFlatMapFunction<String, Integer> {
+		private static final long serialVersionUID = 1L;
 
 		private IntCounter counter = new IntCounter();
 
@@ -376,6 +375,7 @@ public class AccumulatorLiveITCase {
 	 * Outputs format which notifies of accumulator changes and waits for the previous mapper.
 	 */
 	private static class NotifyingOutputFormat implements OutputFormat<Integer> {
+		private static final long serialVersionUID = 1L;
 
 		@Override
 		public void configure(Configuration parameters) {
@@ -439,9 +439,11 @@ public class AccumulatorLiveITCase {
 	}
 
 
-	private static class StreamJobExtractor extends StreamExecutionEnvironment {
-
-		private JobGraph graph = null;
+	/**
+	 * This is used to for creating the example topology. {@link #execute} is never called, we
+	 * only use this to call {@link #getStreamGraph()}.
+	 */
+	private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
 
 		@Override
 		public JobExecutionResult execute() throws Exception {
@@ -450,8 +452,7 @@ public class AccumulatorLiveITCase {
 
 		@Override
 		public JobExecutionResult execute(String jobName) throws Exception {
-			graph = this.streamGraph.getJobGraph();
-			return new JobExecutionResult(new JobID(), -1, null);
+			throw new RuntimeException("This should not be called.");
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
index 04f9f34..76c8e54 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
@@ -39,7 +39,7 @@ public class StreamingProgram {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(host, port, jarFile);
 		env.getConfig().disableSysoutLogging();
 		
-		DataStream<String> text = env.fromElements(WordCountData.TEXT);
+		DataStream<String> text = env.fromElements(WordCountData.TEXT).rebalance();
 
 		DataStream<Word> counts =
 				text.flatMap(new Tokenizer()).groupBy("word").sum("frequency");


[3/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
new file mode 100644
index 0000000..d392fd5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This transformation represents a split of one
+ * {@link org.apache.flink.streaming.api.datastream.DataStream} into several {@code DataStreams}
+ * using an {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}.
+ *
+ * <p>
+ * This does not create a physical operation, it only affects how upstream operations are
+ * connected to downstream operations.
+ *
+ * @param <T> The type of the elements that result from this {@code SplitTransformation}
+ */
+public class SplitTransformation<T> extends StreamTransformation<T> {
+
+	private final StreamTransformation<T> input;
+
+	private final OutputSelector<T> outputSelector;
+
+	/**
+	 * Creates a new {@code SplitTransformation} from the given input and {@code OutputSelector}.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param outputSelector The output selector
+	 */
+	public SplitTransformation(StreamTransformation<T> input,
+			OutputSelector<T> outputSelector) {
+		super("Split", input.getOutputType(), input.getParallelism());
+		this.input = input;
+		this.outputSelector = outputSelector;
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation}.
+	 */
+	public StreamTransformation<T> getInput() {
+		return input;
+	}
+
+	/**
+	 * Returns the {@code OutputSelector}
+	 */
+	public OutputSelector<T> getOutputSelector() {
+		return outputSelector;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
new file mode 100644
index 0000000..dadcfa2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
@@ -0,0 +1,320 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.MissingTypeInfo;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+
+/**
+ * A {@code StreamTransformation} represents the operation that creates a
+ * {@link org.apache.flink.streaming.api.datastream.DataStream}. Every
+ * {@link org.apache.flink.streaming.api.datastream.DataStream} has an underlying
+ * {@code StreamTransformation} that is the origin of said DataStream.
+ *
+ * <p>
+ * API operations such as {@link org.apache.flink.streaming.api.datastream.DataStream#map} create
+ * a tree of {@code StreamTransformation}s underneath. When the stream program is to be executed this
+ * graph is translated to a {@link StreamGraph} using
+ * {@link org.apache.flink.streaming.api.graph.StreamGraphGenerator}.
+ *
+ * <p>
+ * A {@code StreamTransformation} does not necessarily correspond to a physical operation
+ * at runtime. Some operations are only logical concepts. Examples of this are union,
+ * split/select data stream, partitioning.
+ *
+ * <p>
+ * The following graph of {@code StreamTransformations}:
+ *
+ * <pre>
+ *   Source              Source        
+ *      +                   +           
+ *      |                   |           
+ *      v                   v           
+ *  Rebalance          HashPartition    
+ *      +                   +           
+ *      |                   |           
+ *      |                   |           
+ *      +------>Union<------+           
+ *                +                     
+ *                |                     
+ *                v                     
+ *              Split                   
+ *                +                     
+ *                |                     
+ *                v                     
+ *              Select                  
+ *                +                     
+ *                v                     
+ *               Map                    
+ *                +                     
+ *                |                     
+ *                v                     
+ *              Sink 
+ * </pre>
+ *
+ * Would result in this graph of operations at runtime:
+ *
+ * <pre>
+ *  Source              Source
+ *    +                   +
+ *    |                   |
+ *    |                   |
+ *    +------->Map<-------+
+ *              +
+ *              |
+ *              v
+ *             Sink
+ * </pre>
+ *
+ * The information about partitioning, union, split/select end up being encoded in the edges
+ * that connect the sources to the map operation.
+ *
+ * @param <T> The type of the elements that result from this {@code StreamTransformation}
+ */
+public abstract class StreamTransformation<T> {
+
+	// This is used to assign a unique ID to every StreamTransformation
+	protected static Integer idCounter = 0;
+	public static int getNewNodeId() {
+		idCounter++;
+		return idCounter;
+	}
+
+	protected final int id;
+
+	protected String name;
+
+	protected TypeInformation<T> outputType;
+	// This is used to handle MissingTypeInfo. As long as the outputType has not been queried
+	// it can still be changed using setOutputType(). Afterwards an exception is thrown when
+	// trying to change the output type.
+	protected boolean typeUsed;
+
+	private int parallelism;
+
+	protected long bufferTimeout = -1;
+
+	protected StreamGraph.ResourceStrategy resourceStrategy = StreamGraph.ResourceStrategy.DEFAULT;
+
+	/**
+	 * Creates a new {@code StreamTransformation} with the given name, output type and parallelism.
+	 *
+	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
+	 * @param outputType The output type of this {@code StreamTransformation}
+	 * @param parallelism The parallelism of this {@code StreamTransformation}
+	 */
+	public StreamTransformation(String name, TypeInformation<T> outputType, int parallelism) {
+		this.id = getNewNodeId();
+		this.name = Preconditions.checkNotNull(name);
+		this.outputType = outputType;
+		this.parallelism = parallelism;
+	}
+
+	/**
+	 * Returns the unique ID of this {@code StreamTransformation}.
+	 */
+	public int getId() {
+		return id;
+	}
+
+	/**
+	 * Changes the name of this {@code StreamTransformation}.
+	 */
+	public void setName(String name) {
+		this.name = name;
+	}
+
+	/**
+	 * Returns the name of this {@code StreamTransformation}.
+	 */
+	public String getName() {
+		return name;
+	}
+
+	/**
+	 * Returns the parallelism of this {@code StreamTransformation}
+	 */
+	public int getParallelism() {
+		return parallelism;
+	}
+
+	/**
+	 * Sets the parallelism of this {@code StreamTransformation}
+	 * @param parallelism The new parallelism to set on this {@code StreamTransformation}
+	 */
+	public void setParallelism(int parallelism) {
+		Preconditions.checkArgument(parallelism > 0, "Parallelism must be bigger than zero.");
+		this.parallelism = parallelism;
+	}
+
+	/**
+	 * Tries to fill in the type information. Type information can be filled in
+	 * later when the program uses a type hint. This method checks whether the
+	 * type information has ever been accessed before and does not allow
+	 * modifications if the type was accessed already. This ensures consistency
+	 * by making sure different parts of the operation do not assume different
+	 * type information.
+	 *
+	 * @param outputType The type information to fill in.
+	 *
+	 * @throws IllegalStateException Thrown, if the type information has been accessed before.
+	 */
+	public void setOutputType(TypeInformation<T> outputType) {
+		if (typeUsed) {
+			throw new IllegalStateException(
+					"TypeInformation cannot be filled in for the type after it has been used. "
+							+ "Please make sure that the type info hints are the first call after"
+							+ " the transformation function, "
+							+ "before any access to types or semantic properties, etc.");
+		}
+		this.outputType = outputType;
+	}
+
+	/**
+	 * Returns the output type of this {@code StreamTransformation} as a {@link TypeInformation}. Once
+	 * this is used once the output type cannot be changed anymore using {@link #setOutputType}.
+	 *
+	 * @return The output type of this {@code StreamTransformation}
+	 */
+	public TypeInformation<T> getOutputType() {
+		if (outputType instanceof MissingTypeInfo) {
+			MissingTypeInfo typeInfo = (MissingTypeInfo) this.outputType;
+			throw new InvalidTypesException(
+					"The return type of function '"
+							+ typeInfo.getFunctionName()
+							+ "' could not be determined automatically, due to type erasure. "
+							+ "You can give type information hints by using the returns(...) "
+							+ "method on the result of the transformation call, or by letting "
+							+ "your function implement the 'ResultTypeQueryable' "
+							+ "interface.", typeInfo.getTypeException());
+		}
+		typeUsed = true;
+		return this.outputType;
+	}
+
+	/**
+	 * Sets the chaining strategy of this {@code StreamTransformation}.
+	 */
+	public abstract void setChainingStrategy(StreamOperator.ChainingStrategy strategy);
+
+	/**
+	 * Set the buffer timeout of this {@code StreamTransformation}. The timeout is used when
+	 * sending elements over the network. The timeout specifies how long a network buffer
+	 * should be kept waiting before sending. A higher timeout means that more elements will
+	 * be sent in one buffer, this increases throughput. The latency, however, is negatively
+	 * affected by a higher timeout.
+	 */
+	public void setBufferTimeout(long bufferTimeout) {
+		this.bufferTimeout = bufferTimeout;
+	}
+
+	/**
+	 * Returns the buffer timeout of this {@code StreamTransformation}.
+	 *
+	 * <p>
+	 * {@see #setBufferTimeout}
+	 */
+	public long getBufferTimeout() {
+		return bufferTimeout;
+	}
+
+	/**
+	 * Sets the {@link org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy} of this
+	 * {@code StreamTransformation}. The resource strategy is used when scheduling operations on actual
+	 * workers when transforming the StreamTopology to an
+	 * {@link org.apache.flink.runtime.executiongraph.ExecutionGraph}.
+	 */
+	public void setResourceStrategy(StreamGraph.ResourceStrategy resourceStrategy) {
+		this.resourceStrategy = resourceStrategy;
+	}
+
+	/**
+	 * Returns the {@code ResourceStrategy} of this {@code StreamTransformation}.
+	 *
+	 * <p>
+	 * {@see #setResourceStrategy}
+	 */
+	public StreamGraph.ResourceStrategy getResourceStrategy() {
+		return resourceStrategy;
+	}
+
+	/**
+	 * Returns all transitive predecessor {@code StreamTransformation}s of this {@code StreamTransformation}. This
+	 * is, for example, used when determining whether a feedback edge of an iteration
+	 * actually has the iteration head as a predecessor.
+	 *
+	 * @return The list of transitive predecessors.
+	 */
+	public abstract Collection<StreamTransformation<?>> getTransitivePredecessors();
+
+	@Override
+	public String toString() {
+		return getClass().getSimpleName() + "{" +
+				"id=" + id +
+				", name='" + name + '\'' +
+				", outputType=" + outputType +
+				", parallelism=" + parallelism +
+				'}';
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (!(o instanceof StreamTransformation)) {
+			return false;
+		}
+
+		StreamTransformation<?> that = (StreamTransformation<?>) o;
+
+		if (bufferTimeout != that.bufferTimeout) {
+			return false;
+		}
+		if (id != that.id) {
+			return false;
+		}
+		if (parallelism != that.parallelism) {
+			return false;
+		}
+		if (!name.equals(that.name)) {
+			return false;
+		}
+		if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) {
+			return false;
+		}
+		return resourceStrategy == that.resourceStrategy;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = id;
+		result = 31 * result + name.hashCode();
+		result = 31 * result + (outputType != null ? outputType.hashCode() : 0);
+		result = 31 * result + parallelism;
+		result = 31 * result + (int) (bufferTimeout ^ (bufferTimeout >>> 32));
+		result = 31 * result + resourceStrategy.hashCode();
+		return result;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
new file mode 100644
index 0000000..e7273c5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to two input
+ * {@code StreamTransformations}. The result is again only one stream.
+ *
+ * @param <IN1> The type of the elements in the first input {@code StreamTransformation}
+ * @param <IN2> The type of the elements in the second input {@code StreamTransformation}
+ * @param <OUT> The type of the elements that result from this {@code TwoInputTransformation}
+ */
+public class TwoInputTransformation<IN1, IN2, OUT> extends StreamTransformation<OUT> {
+
+	private final StreamTransformation<IN1> input1;
+	private final StreamTransformation<IN2> input2;
+
+	private final TwoInputStreamOperator<IN1, IN2, OUT> operator;
+
+	/**
+	 * Creates a new {@code TwoInputTransformation} from the given inputs and operator.
+	 *
+	 * @param input1 The first input {@code StreamTransformation}
+	 * @param input2 The second input {@code StreamTransformation}
+	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
+	 * @param operator The {@code TwoInputStreamOperator}
+	 * @param outputType The type of the elements produced by this Transformation
+	 * @param parallelism The parallelism of this Transformation
+	 */
+	public TwoInputTransformation(
+			StreamTransformation<IN1> input1,
+			StreamTransformation<IN2> input2,
+			String name,
+			TwoInputStreamOperator<IN1, IN2, OUT> operator,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.input1 = input1;
+		this.input2 = input2;
+		this.operator = operator;
+	}
+
+	/**
+	 * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}.
+	 */
+	public StreamTransformation<IN1> getInput1() {
+		return input1;
+	}
+
+	/**
+	 * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}.
+	 */
+	public StreamTransformation<IN2> getInput2() {
+		return input2;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the first input.
+	 */
+	public TypeInformation<IN1> getInputType1() {
+		return input1.getOutputType();
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements from the first input.
+	 */
+	public TypeInformation<IN2> getInputType2() {
+		return input2.getOutputType();
+	}
+
+	/**
+	 * Returns the {@code TwoInputStreamOperator} of this Transformation.
+	 */
+	public TwoInputStreamOperator<IN1, IN2, OUT> getOperator() {
+		return operator;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input1.getTransitivePredecessors());
+		result.addAll(input2.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		operator.setChainingStrategy(strategy);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
new file mode 100644
index 0000000..4fa3c0a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This transformation represents a union of several input
+ * {@link StreamTransformation StreamTransformations}.
+ *
+ * <p>
+ * This does not create a physical operation, it only affects how upstream operations are
+ * connected to downstream operations.
+ *
+ * @param <T> The type of the elements that result from this {@code UnionTransformation}
+ */
+public class UnionTransformation<T> extends StreamTransformation<T> {
+	private final List<StreamTransformation<T>> inputs;
+
+	/**
+	 * Creates a new {@code UnionTransformation} from the given input {@code StreamTransformations}.
+	 *
+	 * <p>
+	 * The input {@code StreamTransformations} must all have the same type.
+	 *
+	 * @param inputs The list of input {@code StreamTransformations}
+	 */
+	public UnionTransformation(List<StreamTransformation<T>> inputs) {
+		super("Union", inputs.get(0).getOutputType(), inputs.get(0).getParallelism());
+
+		for (StreamTransformation<T> input: inputs) {
+			if (!input.getOutputType().equals(getOutputType())) {
+				throw new UnsupportedOperationException("Type mismatch in input " + input);
+			}
+		}
+
+		this.inputs = Lists.newArrayList(inputs);
+	}
+
+	/**
+	 * Returns the list of input {@code StreamTransformations}.
+	 */
+	public List<StreamTransformation<T>> getInputs() {
+		return inputs;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		for (StreamTransformation<T> input: inputs) {
+			result.addAll(input.getTransitivePredecessors());
+		}
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
index f51a04f..f3d851c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
@@ -23,8 +23,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 /**
  * Partitioner that selects all the output channels.
  *
- * @param <T>
- *            Type of the Tuple
+ * @param <T> Type of the elements in the Stream being broadcast
  */
 public class BroadcastPartitioner<T> extends StreamPartitioner<T> {
 	private static final long serialVersionUID = 1L;
@@ -33,10 +32,6 @@ public class BroadcastPartitioner<T> extends StreamPartitioner<T> {
 	boolean set;
 	int setNumber;
 
-	public BroadcastPartitioner() {
-		super(PartitioningStrategy.BROADCAST);
-	}
-
 	@Override
 	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
 			int numberOfOutputChannels) {
@@ -52,4 +47,14 @@ public class BroadcastPartitioner<T> extends StreamPartitioner<T> {
 			return returnArray;
 		}
 	}
+
+	@Override
+	public StreamPartitioner<T> copy() {
+		return this;
+	}
+
+	@Override
+	public String toString() {
+		return "BROADCAST";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
index 6c40c03..7bb9480 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
@@ -38,7 +38,6 @@ public class CustomPartitionerWrapper<K, T> extends StreamPartitioner<T> {
 	KeySelector<T, K> keySelector;
 
 	public CustomPartitionerWrapper(Partitioner<K> partitioner, KeySelector<T, K> keySelector) {
-		super(PartitioningStrategy.CUSTOM);
 		this.partitioner = partitioner;
 		this.keySelector = keySelector;
 	}
@@ -58,4 +57,14 @@ public class CustomPartitionerWrapper<K, T> extends StreamPartitioner<T> {
 
 		return returnArray;
 	}
+
+	@Override
+	public StreamPartitioner<T> copy() {
+		return this;
+	}
+
+	@Override
+	public String toString() {
+		return "CUSTOM";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java
deleted file mode 100644
index 7026d45..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/FieldsPartitioner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that selects the same (one) channel for two Tuples having a
- * specified fields equal.
- * 
- * @param <T>
- *            Type of the Tuple
- */
-public class FieldsPartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[1];
-	KeySelector<T, ?> keySelector;
-
-	public FieldsPartitioner(KeySelector<T, ?> keySelector) {
-		super(PartitioningStrategy.GROUPBY);
-		this.keySelector = keySelector;
-	}
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		Object key;
-		try {
-			key = keySelector.getKey(record.getInstance().getValue());
-		} catch (Exception e) {
-			throw new RuntimeException("Could not extract key from " + record.getInstance().getValue(), e);
-		}
-		returnArray[0] = Math.abs(key.hashCode() % numberOfOutputChannels);
-
-		return returnArray;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
new file mode 100644
index 0000000..4fb460c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.partitioner;
+
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * Partitioner that forwards elements only to the locally running downstream operation.
+ * 
+ * @param <T> Type of the elements in the Stream
+ */
+public class ForwardPartitioner<T> extends StreamPartitioner<T> {
+	private static final long serialVersionUID = 1L;
+
+	private int[] returnArray = new int[] {0};
+
+	@Override
+	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record, int numberOfOutputChannels) {
+		return returnArray;
+	}
+	
+	public StreamPartitioner<T> copy() {
+		return this;
+	}
+	
+	@Override
+	public String toString() {
+		return "FORWARD";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
index 46b290b..b19fb41 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
@@ -20,19 +20,29 @@ package org.apache.flink.streaming.runtime.partitioner;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
-//Group to the partitioner with the lowest id
+/**
+ * Partitioner that sends all elements to the downstream operator with subtask ID=0;
+ *
+ * @param <T> Type of the elements in the Stream being partitioned
+ */
 public class GlobalPartitioner<T> extends StreamPartitioner<T> {
 	private static final long serialVersionUID = 1L;
 
 	private int[] returnArray = new int[] { 0 };
 
-	public GlobalPartitioner() {
-		super(PartitioningStrategy.GLOBAL);
-	}
-
 	@Override
 	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
 			int numberOfOutputChannels) {
 		return returnArray;
 	}
+
+	@Override
+	public StreamPartitioner<T> copy() {
+		return this;
+	}
+
+	@Override
+	public String toString() {
+		return "GLOBAL";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
new file mode 100644
index 0000000..a3f5158
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.partitioner;
+
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * Partitioner selects the target channel based on the hash value of a key from a
+ * {@link KeySelector}.
+ *
+ * @param <T> Type of the elements in the Stream being partitioned
+ */
+public class HashPartitioner<T> extends StreamPartitioner<T> {
+	private static final long serialVersionUID = 1L;
+
+	private int[] returnArray = new int[1];
+	KeySelector<T, ?> keySelector;
+
+	public HashPartitioner(KeySelector<T, ?> keySelector) {
+		this.keySelector = keySelector;
+	}
+
+	@Override
+	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
+			int numberOfOutputChannels) {
+		Object key;
+		try {
+			key = keySelector.getKey(record.getInstance().getValue());
+		} catch (Exception e) {
+			throw new RuntimeException("Could not extract key from " + record.getInstance().getValue(), e);
+		}
+		returnArray[0] = Math.abs(key.hashCode() % numberOfOutputChannels);
+
+		return returnArray;
+	}
+
+	@Override
+	public StreamPartitioner<T> copy() {
+		return this;
+	}
+
+	@Override
+	public String toString() {
+		return "HASH";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
index e6ad821..2dfff0e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
@@ -24,34 +24,26 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
  * Partitioner that distributes the data equally by cycling through the output
  * channels.
  * 
- * @param <T>
- *            Type of the Tuple
+ * @param <T> Type of the elements in the Stream being rebalanced
  */
 public class RebalancePartitioner<T> extends StreamPartitioner<T> {
 	private static final long serialVersionUID = 1L;
 
 	private int[] returnArray = new int[] {-1};
-	private boolean forward;
-
-	public RebalancePartitioner(boolean forward) {
-		super(forward ? PartitioningStrategy.FORWARD : PartitioningStrategy.DISTRIBUTE);
-		this.forward = forward;
-	}
 
 	@Override
 	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
 			int numberOfOutputChannels) {
 		this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels;
-
 		return this.returnArray;
 	}
 	
 	public StreamPartitioner<T> copy() {
-		return new RebalancePartitioner<T>(forward);
+		return this;
 	}
 	
 	@Override
 	public String toString() {
-		return forward ? "ForwardPartitioner" : "RebalancePartitioner";
+		return "REBALANCE";
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
index ba50113..93c6f9c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
@@ -36,14 +36,20 @@ public class ShufflePartitioner<T> extends StreamPartitioner<T> {
 
 	private int[] returnArray = new int[1];
 
-	public ShufflePartitioner() {
-		super(PartitioningStrategy.SHUFFLE);
-	}
-
 	@Override
 	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
 			int numberOfOutputChannels) {
 		returnArray[0] = random.nextInt(numberOfOutputChannels);
 		return returnArray;
 	}
+
+	@Override
+	public StreamPartitioner<T> copy() {
+		return new ShufflePartitioner<T>();
+	}
+
+	@Override
+	public String toString() {
+		return "SHUFFLE";
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
index b37655b..4ef360d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
@@ -24,30 +24,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 public abstract class StreamPartitioner<T> implements
 		ChannelSelector<SerializationDelegate<StreamRecord<T>>>, Serializable {
-
-	public enum PartitioningStrategy {
-
-		FORWARD, DISTRIBUTE, SHUFFLE, BROADCAST, GLOBAL, GROUPBY, CUSTOM
-
-	}
-
 	private static final long serialVersionUID = 1L;
-	private PartitioningStrategy strategy;
-
-	public StreamPartitioner(PartitioningStrategy strategy) {
-		this.strategy = strategy;
-	}
-
-	public PartitioningStrategy getStrategy() {
-		return strategy;
-	}
 
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-	
-	@Override
-	public String toString() {
-		return this.getClass().getSimpleName();
-	}
+	public abstract StreamPartitioner<T> copy();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
index 3ad6b8e..fdf7697 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
@@ -24,19 +24,23 @@ import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.util.NoOpSink;
+import org.apache.flink.streaming.util.ReceiveCheckNoOpSink;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
 
 @SuppressWarnings("serial")
-public class ChainedRuntimeContextTest {
-	private static final long MEMORYSIZE = 32;
+public class ChainedRuntimeContextTest extends StreamingMultipleProgramsTestBase {
 	private static RuntimeContext srcContext;
 	private static RuntimeContext mapContext;
 
 	@Test
 	public void test() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
-		env.addSource(new TestSource()).map(new TestMap());
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		env.addSource(new TestSource()).map(new TestMap()).addSink(new NoOpSink<Integer>());
 		env.execute();
 
 		assertNotEquals(srcContext, mapContext);

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
index 369b384..7ea1309 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
@@ -31,21 +31,21 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
-public class CoStreamTest {
-
-	private static final long MEMORY_SIZE = 32;
+public class CoStreamTest extends StreamingMultipleProgramsTestBase {
 
 	private static ArrayList<String> expected = new ArrayList<String>();
 
 	@Test
 	public void test() {
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 
 		TestListResultSink<String> resultSink = new TestListResultSink<String>();
 
@@ -129,4 +129,4 @@ public class CoStreamTest {
 
 		assertEquals(expected, result);
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 324143f..9775392 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -54,19 +54,20 @@ import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.windowing.helper.Count;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
-import org.apache.flink.streaming.runtime.partitioner.FieldsPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.NoOpSink;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
-public class DataStreamTest {
+public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
-	private static final long MEMORYSIZE = 32;
-	private static int PARALLELISM = 2;
 
 	/**
 	 * Tests {@link SingleOutputStreamOperator#name(String)} functionality.
@@ -75,7 +76,7 @@ public class DataStreamTest {
 	 */
 	@Test
 	public void testNaming() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream<Long> dataStream1 = env.generateSequence(0, 0).name("testSource1")
 				.map(new MapFunction<Long, Long>() {
@@ -93,7 +94,7 @@ public class DataStreamTest {
 					}
 				}).name("testMap");
 
-		DataStream<Long> connected = dataStream1.connect(dataStream2)
+		DataStreamSink<Long> connected = dataStream1.connect(dataStream2)
 				.flatMap(new CoFlatMapFunction<Long, Long, Long>() {
 					@Override
 					public void flatMap1(Long value, Collector<Long> out) throws Exception {
@@ -110,7 +111,8 @@ public class DataStreamTest {
 						return null;
 					}
 				}).name("testWindowFold")
-				.flatten();
+				.flatten()
+				.print();
 
 		//test functionality through the operator names in the execution plan
 		String plan = env.getExecutionPlan();
@@ -130,8 +132,7 @@ public class DataStreamTest {
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testPartitioning() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
-		StreamGraph graph = env.getStreamGraph();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream src1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
 		DataStream src2 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
@@ -143,10 +144,15 @@ public class DataStreamTest {
 		DataStream group3 = src1.groupBy("f0");
 		DataStream group4 = src1.groupBy(new FirstSelector());
 
-		assertTrue(isPartitioned(graph.getStreamEdge(group1.getId(), createDownStreamId(group1))));
-		assertTrue(isPartitioned(graph.getStreamEdge(group2.getId(), createDownStreamId(group2))));
-		assertTrue(isPartitioned(graph.getStreamEdge(group3.getId(), createDownStreamId(group3))));
-		assertTrue(isPartitioned(graph.getStreamEdge(group4.getId(), createDownStreamId(group4))));
+		int id1 = createDownStreamId(group1);
+		int id2 = createDownStreamId(group2);
+		int id3 = createDownStreamId(group3);
+		int id4 = createDownStreamId(group4);
+
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id4)));
 
 		assertTrue(isGrouped(group1));
 		assertTrue(isGrouped(group2));
@@ -159,10 +165,15 @@ public class DataStreamTest {
 		DataStream partition3 = src1.partitionByHash("f0");
 		DataStream partition4 = src1.partitionByHash(new FirstSelector());
 
-		assertTrue(isPartitioned(graph.getStreamEdge(partition1.getId(), createDownStreamId(partition1))));
-		assertTrue(isPartitioned(graph.getStreamEdge(partition2.getId(), createDownStreamId(partition2))));
-		assertTrue(isPartitioned(graph.getStreamEdge(partition3.getId(), createDownStreamId(partition3))));
-		assertTrue(isPartitioned(graph.getStreamEdge(partition4.getId(), createDownStreamId(partition4))));
+		int pid1 = createDownStreamId(partition1);
+		int pid2 = createDownStreamId(partition2);
+		int pid3 = createDownStreamId(partition3);
+		int pid4 = createDownStreamId(partition4);
+
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid4)));
 
 		assertFalse(isGrouped(partition1));
 		assertFalse(isGrouped(partition3));
@@ -181,9 +192,13 @@ public class DataStreamTest {
 		DataStream customPartition3 = src1.partitionCustom(longPartitioner, "f0");
 		DataStream customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector());
 
-		assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition1.getId(), createDownStreamId(customPartition1))));
-		assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition3.getId(), createDownStreamId(customPartition3))));
-		assertTrue(isCustomPartitioned(graph.getStreamEdge(customPartition4.getId(), createDownStreamId(customPartition4))));
+		int cid1 = createDownStreamId(customPartition1);
+		int cid2 = createDownStreamId(customPartition3);
+		int cid3 = createDownStreamId(customPartition4);
+
+		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid1)));
+		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid2)));
+		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid3)));
 
 		assertFalse(isGrouped(customPartition1));
 		assertFalse(isGrouped(customPartition3));
@@ -205,20 +220,20 @@ public class DataStreamTest {
 		ConnectedDataStream connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector());
 		Integer downStreamId5 = createDownStreamId(connectedGroup5);
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup1.getFirst().getId(), downStreamId1)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup1.getSecond().getId(), downStreamId1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId1)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup2.getFirst().getId(), downStreamId2)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup2.getSecond().getId(), downStreamId2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId2)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup3.getFirst().getId(), downStreamId3)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup3.getSecond().getId(), downStreamId3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId3)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup4.getFirst().getId(), downStreamId4)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup4.getSecond().getId(), downStreamId4)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId4)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId4)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup5.getFirst().getId(), downStreamId5)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedGroup5.getSecond().getId(), downStreamId5)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId5)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId5)));
 
 		assertTrue(isGrouped(connectedGroup1));
 		assertTrue(isGrouped(connectedGroup2));
@@ -242,20 +257,30 @@ public class DataStreamTest {
 		ConnectedDataStream connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
 		Integer connectDownStreamId5 = createDownStreamId(connectedPartition5);
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition1.getFirst().getId(), connectDownStreamId1)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition1.getSecond().getId(), connectDownStreamId1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
+				connectDownStreamId1)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
+				connectDownStreamId1)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition2.getFirst().getId(), connectDownStreamId2)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition2.getSecond().getId(), connectDownStreamId2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
+				connectDownStreamId2)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
+				connectDownStreamId2)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition3.getFirst().getId(), connectDownStreamId3)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition3.getSecond().getId(), connectDownStreamId3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
+				connectDownStreamId3)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
+				connectDownStreamId3)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition4.getFirst().getId(), connectDownStreamId4)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition4.getSecond().getId(), connectDownStreamId4)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
+				connectDownStreamId4)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
+				connectDownStreamId4)));
 
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition5.getFirst().getId(), connectDownStreamId5)));
-		assertTrue(isPartitioned(graph.getStreamEdge(connectedPartition5.getSecond().getId(), connectDownStreamId5)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
+				connectDownStreamId5)));
+		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
+				connectDownStreamId5)));
 
 		assertFalse(isGrouped(connectedPartition1));
 		assertFalse(isGrouped(connectedPartition2));
@@ -269,17 +294,17 @@ public class DataStreamTest {
 	 */
 	@Test
 	public void testParallelism() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(10, MEMORYSIZE);
-		StreamGraph graph = env.getStreamGraph();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStreamSource<Tuple2<Long, Long>> src = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
+		env.setParallelism(10);
 
 		SingleOutputStreamOperator<Long, ?> map = src.map(new MapFunction<Tuple2<Long, Long>, Long>() {
 			@Override
 			public Long map(Tuple2<Long, Long> value) throws Exception {
 				return null;
 			}
-		});
+		}).name("MyMap");
 
 		DataStream<Long> windowed = map
 				.window(Count.of(10))
@@ -288,7 +313,10 @@ public class DataStreamTest {
 					public Long fold(Long accumulator, Long value) throws Exception {
 						return null;
 					}
-				}).flatten();
+				})
+				.flatten();
+
+		windowed.addSink(new NoOpSink<Long>());
 
 		DataStreamSink<Long> sink = map.addSink(new SinkFunction<Long>() {
 			@Override
@@ -296,16 +324,21 @@ public class DataStreamTest {
 			}
 		});
 
-		assertEquals(1, graph.getStreamNode(src.getId()).getParallelism());
-		assertEquals(10, graph.getStreamNode(map.getId()).getParallelism());
-		assertEquals(10, graph.getStreamNode(windowed.getId()).getParallelism());
-		assertEquals(10, graph.getStreamNode(sink.getId()).getParallelism());
+		assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism());
+		assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
+		assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism());
+		assertEquals(10,
+				env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
 
 		env.setParallelism(7);
-		assertEquals(1, graph.getStreamNode(src.getId()).getParallelism());
-		assertEquals(7, graph.getStreamNode(map.getId()).getParallelism());
-		assertEquals(7, graph.getStreamNode(windowed.getId()).getParallelism());
-		assertEquals(7, graph.getStreamNode(sink.getId()).getParallelism());
+
+		// Some parts, such as windowing rely on the fact that previous operators have a parallelism
+		// set when instantiating the Discretizer. This would break if we dynamically changed
+		// the parallelism of operations when changing the setting on the Execution Environment.
+		assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism());
+		assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
+		assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism());
+		assertEquals(10, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
 
 		try {
 			src.setParallelism(3);
@@ -314,21 +347,22 @@ public class DataStreamTest {
 		}
 
 		DataStreamSource<Long> parallelSource = env.generateSequence(0, 0);
-		assertEquals(7, graph.getStreamNode(parallelSource.getId()).getParallelism());
+		parallelSource.addSink(new NoOpSink<Long>());
+		assertEquals(7, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism());
 
 		parallelSource.setParallelism(3);
-		assertEquals(3, graph.getStreamNode(parallelSource.getId()).getParallelism());
+		assertEquals(3, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism());
 
 		map.setParallelism(2);
-		assertEquals(2, graph.getStreamNode(map.getId()).getParallelism());
+		assertEquals(2, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
 
 		sink.setParallelism(4);
-		assertEquals(4, graph.getStreamNode(sink.getId()).getParallelism());
+		assertEquals(4, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
 	}
 
 	@Test
 	public void testTypeInfo() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream<Long> src1 = env.generateSequence(0, 0);
 		assertEquals(TypeExtractor.getForClass(Long.class), src1.getType());
@@ -366,9 +400,7 @@ public class DataStreamTest {
 
 	@Test
 	public void operatorTest() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
-
-		StreamGraph streamGraph = env.getStreamGraph();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStreamSource<Long> src = env.generateSequence(0, 0);
 
@@ -379,6 +411,7 @@ public class DataStreamTest {
 			}
 		};
 		DataStream<Integer> map = src.map(mapFunction);
+		map.addSink(new NoOpSink<Integer>());
 		assertEquals(mapFunction, getFunctionForDataStream(map));
 
 
@@ -388,6 +421,7 @@ public class DataStreamTest {
 			}
 		};
 		DataStream<Integer> flatMap = src.flatMap(flatMapFunction);
+		flatMap.addSink(new NoOpSink<Integer>());
 		assertEquals(flatMapFunction, getFunctionForDataStream(flatMap));
 
 		FilterFunction<Integer> filterFunction = new FilterFunction<Integer>() {
@@ -401,16 +435,18 @@ public class DataStreamTest {
 				.union(flatMap)
 				.filter(filterFunction);
 
+		unionFilter.addSink(new NoOpSink<Integer>());
+
 		assertEquals(filterFunction, getFunctionForDataStream(unionFilter));
 
 		try {
-			streamGraph.getStreamEdge(map.getId(), unionFilter.getId());
+			env.getStreamGraph().getStreamEdge(map.getId(), unionFilter.getId());
 		} catch (RuntimeException e) {
 			fail(e.getMessage());
 		}
 
 		try {
-			streamGraph.getStreamEdge(flatMap.getId(), unionFilter.getId());
+			env.getStreamGraph().getStreamEdge(flatMap.getId(), unionFilter.getId());
 		} catch (RuntimeException e) {
 			fail(e.getMessage());
 		}
@@ -423,14 +459,15 @@ public class DataStreamTest {
 		};
 
 		SplitDataStream<Integer> split = unionFilter.split(outputSelector);
-		List<OutputSelector<?>> outputSelectors = streamGraph.getStreamNode(split.getId()).getOutputSelectors();
+		split.select("dummy").addSink(new NoOpSink<Integer>());
+		List<OutputSelector<?>> outputSelectors = env.getStreamGraph().getStreamNode(unionFilter.getId()).getOutputSelectors();
 		assertEquals(1, outputSelectors.size());
 		assertEquals(outputSelector, outputSelectors.get(0));
 
 		DataStream<Integer> select = split.select("a");
 		DataStreamSink<Integer> sink = select.print();
 
-		StreamEdge splitEdge = streamGraph.getStreamEdge(select.getId(), sink.getId());
+		StreamEdge splitEdge = env.getStreamGraph().getStreamEdge(unionFilter.getId(), sink.getTransformation().getId());
 		assertEquals("a", splitEdge.getSelectedNames().get(0));
 
 		ConnectedDataStream<Integer, Integer> connect = map.connect(flatMap);
@@ -446,16 +483,17 @@ public class DataStreamTest {
 			}
 		};
 		DataStream<String> coMap = connect.map(coMapper);
+		coMap.addSink(new NoOpSink<String>());
 		assertEquals(coMapper, getFunctionForDataStream(coMap));
 
 		try {
-			streamGraph.getStreamEdge(map.getId(), coMap.getId());
+			env.getStreamGraph().getStreamEdge(map.getId(), coMap.getId());
 		} catch (RuntimeException e) {
 			fail(e.getMessage());
 		}
 
 		try {
-			streamGraph.getStreamEdge(flatMap.getId(), coMap.getId());
+			env.getStreamGraph().getStreamEdge(flatMap.getId(), coMap.getId());
 		} catch (RuntimeException e) {
 			fail(e.getMessage());
 		}
@@ -463,12 +501,11 @@ public class DataStreamTest {
 	
 	@Test
 	public void sinkKeyTest() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
-		StreamGraph streamGraph = env.getStreamGraph();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		DataStream<Long> sink = env.generateSequence(1, 100).print();
-		assertTrue(streamGraph.getStreamNode(sink.getId()).getStatePartitioner() == null);
-		assertTrue(streamGraph.getStreamNode(sink.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner);
+		DataStreamSink<Long> sink = env.generateSequence(1, 100).print();
+		assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getStatePartitioner() == null);
+		assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof ForwardPartitioner);
 
 		KeySelector<Long, Long> key1 = new KeySelector<Long, Long>() {
 
@@ -480,11 +517,11 @@ public class DataStreamTest {
 			}
 		};
 
-		DataStream<Long> sink2 = env.generateSequence(1, 100).keyBy(key1).print();
+		DataStreamSink<Long> sink2 = env.generateSequence(1, 100).keyBy(key1).print();
 
-		assertTrue(streamGraph.getStreamNode(sink2.getId()).getStatePartitioner() != null);
-		assertEquals(key1, streamGraph.getStreamNode(sink2.getId()).getStatePartitioner());
-		assertTrue(streamGraph.getStreamNode(sink2.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner);
+		assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner() != null);
+		assertEquals(key1, env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner());
+		assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner);
 
 		KeySelector<Long, Long> key2 = new KeySelector<Long, Long>() {
 
@@ -496,49 +533,52 @@ public class DataStreamTest {
 			}
 		};
 
-		DataStream<Long> sink3 = env.generateSequence(1, 100).keyBy(key2).print();
+		DataStreamSink<Long> sink3 = env.generateSequence(1, 100).keyBy(key2).print();
 
-		assertTrue(streamGraph.getStreamNode(sink3.getId()).getStatePartitioner() != null);
-		assertEquals(key2, streamGraph.getStreamNode(sink3.getId()).getStatePartitioner());
-		assertTrue(streamGraph.getStreamNode(sink3.getId()).getInEdges().get(0).getPartitioner() instanceof FieldsPartitioner);
+		assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner() != null);
+		assertEquals(key2, env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner());
+		assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner);
 	}
 
 	@Test
 	public void testChannelSelectors() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
-
-		StreamGraph streamGraph = env.getStreamGraph();
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStreamSource<Long> src = env.generateSequence(0, 0);
 
 		DataStream<Long> broadcast = src.broadcast();
 		DataStreamSink<Long> broadcastSink = broadcast.print();
 		StreamPartitioner<?> broadcastPartitioner =
-				streamGraph.getStreamEdge(broadcast.getId(), broadcastSink.getId()).getPartitioner();
+				env.getStreamGraph().getStreamEdge(src.getId(),
+						broadcastSink.getTransformation().getId()).getPartitioner();
 		assertTrue(broadcastPartitioner instanceof BroadcastPartitioner);
 
 		DataStream<Long> shuffle = src.shuffle();
 		DataStreamSink<Long> shuffleSink = shuffle.print();
 		StreamPartitioner<?> shufflePartitioner =
-				streamGraph.getStreamEdge(shuffle.getId(), shuffleSink.getId()).getPartitioner();
+				env.getStreamGraph().getStreamEdge(src.getId(),
+						shuffleSink.getTransformation().getId()).getPartitioner();
 		assertTrue(shufflePartitioner instanceof ShufflePartitioner);
 
 		DataStream<Long> forward = src.forward();
 		DataStreamSink<Long> forwardSink = forward.print();
 		StreamPartitioner<?> forwardPartitioner =
-				streamGraph.getStreamEdge(forward.getId(), forwardSink.getId()).getPartitioner();
-		assertTrue(forwardPartitioner instanceof RebalancePartitioner);
+				env.getStreamGraph().getStreamEdge(src.getId(),
+						forwardSink.getTransformation().getId()).getPartitioner();
+		assertTrue(forwardPartitioner instanceof ForwardPartitioner);
 
 		DataStream<Long> rebalance = src.rebalance();
 		DataStreamSink<Long> rebalanceSink = rebalance.print();
 		StreamPartitioner<?> rebalancePartitioner =
-				streamGraph.getStreamEdge(rebalance.getId(), rebalanceSink.getId()).getPartitioner();
+				env.getStreamGraph().getStreamEdge(src.getId(),
+						rebalanceSink.getTransformation().getId()).getPartitioner();
 		assertTrue(rebalancePartitioner instanceof RebalancePartitioner);
 
 		DataStream<Long> global = src.global();
 		DataStreamSink<Long> globalSink = global.print();
 		StreamPartitioner<?> globalPartitioner =
-				streamGraph.getStreamEdge(global.getId(), globalSink.getId()).getPartitioner();
+				env.getStreamGraph().getStreamEdge(src.getId(),
+						globalSink.getTransformation().getId()).getPartitioner();
 		assertTrue(globalPartitioner instanceof GlobalPartitioner);
 	}
 
@@ -559,7 +599,7 @@ public class DataStreamTest {
 	}
 
 	private static Integer createDownStreamId(DataStream dataStream) {
-		return dataStream.print().getId();
+		return dataStream.print().getTransformation().getId();
 	}
 
 	private static boolean isGrouped(DataStream dataStream) {
@@ -567,7 +607,7 @@ public class DataStreamTest {
 	}
 
 	private static Integer createDownStreamId(ConnectedDataStream dataStream) {
-		return dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
+		SingleOutputStreamOperator coMap = dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
 			@Override
 			public Object map1(Tuple2<Long, Long> value) {
 				return null;
@@ -577,7 +617,9 @@ public class DataStreamTest {
 			public Object map2(Tuple2<Long, Long> value) {
 				return null;
 			}
-		}).getId();
+		});
+		coMap.addSink(new NoOpSink());
+		return coMap.getId();
 	}
 
 	private static boolean isGrouped(ConnectedDataStream dataStream) {
@@ -585,7 +627,7 @@ public class DataStreamTest {
 	}
 
 	private static boolean isPartitioned(StreamEdge edge) {
-		return edge.getPartitioner() instanceof FieldsPartitioner;
+		return edge.getPartitioner() instanceof HashPartitioner;
 	}
 
 	private static boolean isCustomPartitioned(StreamEdge edge) {


[2/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 0fad3dd..285ee57 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -1,27 +1,22 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
 
 package org.apache.flink.streaming.api;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -31,146 +26,235 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.IterativeDataStream;
 import org.apache.flink.streaming.api.datastream.IterativeDataStream.ConnectedIterativeDataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.graph.StreamEdge;
 import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamLoop;
 import org.apache.flink.streaming.api.graph.StreamNode;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.util.EvenOddOutputSelector;
+import org.apache.flink.streaming.util.NoOpIntMap;
+import org.apache.flink.streaming.util.ReceiveCheckNoOpSink;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
+import static org.junit.Assert.*;
+
 @SuppressWarnings({ "unchecked", "unused", "serial" })
 public class IterateTest extends StreamingMultipleProgramsTestBase {
 
-	private static final long MEMORYSIZE = 32;
 	private static boolean iterated[];
-	private static int PARALLELISM = 2;
 
-	@Test
-	public void testException() throws Exception {
+	@Test(expected = UnsupportedOperationException.class)
+	public void testIncorrectParallelism() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
 		DataStream<Integer> source = env.fromElements(1, 10);
+
+		IterativeDataStream<Integer> iter1 = source.iterate();
+		SingleOutputStreamOperator<Integer, ?> map1 = iter1.map(NoOpIntMap);
+		iter1.closeWith(map1).print();
+	}
+
+	@Test
+	public void testDoubleClosing() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// introduce dummy mapper to get to correct parallelism
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+
 		IterativeDataStream<Integer> iter1 = source.iterate();
-		IterativeDataStream<Integer> iter2 = source.iterate();
 
 		iter1.closeWith(iter1.map(NoOpIntMap));
-		// Check for double closing
-		try {
-			iter1.closeWith(iter1.map(NoOpIntMap));
-			fail();
-		} catch (Exception e) {
-		}
+		iter1.closeWith(iter1.map(NoOpIntMap));
+	}
 
-		// Check for closing iteration without head
-		try {
-			iter2.closeWith(iter1.map(NoOpIntMap));
-			fail();
-		} catch (Exception e) {
-		}
 
-		iter2.map(NoOpIntMap);
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDifferingParallelism() throws Exception {
 
-		// Check for executing with empty iteration
-		try {
-			env.execute();
-			fail();
-		} catch (Exception e) {
-		}
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// introduce dummy mapper to get to correct parallelism
+		DataStream<Integer> source = env.fromElements(1, 10)
+				.map(NoOpIntMap);
+
+		IterativeDataStream<Integer> iter1 = source.iterate();
+
+
+		iter1.closeWith(iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2));
+
+	}
+
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testCoDifferingParallelism() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// introduce dummy mapper to get to correct parallelism
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+
+		ConnectedIterativeDataStream<Integer, Integer> coIter = source.iterate().withFeedbackType(
+				Integer.class);
+
+
+		coIter.closeWith(coIter.map(NoOpIntCoMap).setParallelism(DEFAULT_PARALLELISM / 2));
+
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testClosingFromOutOfLoop() throws Exception {
+
+		// this test verifies that we cannot close an iteration with a DataStream that does not
+		// have the iteration in its predecessors
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// introduce dummy mapper to get to correct parallelism
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+
+		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeDataStream<Integer> iter2 = source.iterate();
+
+
+		iter2.closeWith(iter1.map(NoOpIntMap));
+
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testCoIterClosingFromOutOfLoop() throws Exception {
+
+		// this test verifies that we cannot close an iteration with a DataStream that does not
+		// have the iteration in its predecessors
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// introduce dummy mapper to get to correct parallelism
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+
+		IterativeDataStream<Integer> iter1 = source.iterate();
+		ConnectedIterativeDataStream<Integer, Integer> coIter = source.iterate().withFeedbackType(
+				Integer.class);
+
+
+		coIter.closeWith(iter1.map(NoOpIntMap));
+
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testExecutionWithEmptyIteration() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+
+		IterativeDataStream<Integer> iter1 = source.iterate();
+
+		iter1.map(NoOpIntMap).print();
+
+		env.execute();
 	}
 
 	@Test
 	public void testImmutabilityWithCoiteration() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
-		DataStream<Integer> source = env.fromElements(1, 10);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap); // for rebalance
 
 		IterativeDataStream<Integer> iter1 = source.iterate();
 		// Calling withFeedbackType should create a new iteration
 		ConnectedIterativeDataStream<Integer, String> iter2 = iter1.withFeedbackType(String.class);
 
-		iter1.closeWith(iter1.map(NoOpIntMap));
-		iter2.closeWith(iter2.map(NoOpCoMap));
+		iter1.closeWith(iter1.map(NoOpIntMap)).print();
+		iter2.closeWith(iter2.map(NoOpCoMap)).print();
 
 		StreamGraph graph = env.getStreamGraph();
 
-		graph.getJobGraph();
+		assertEquals(2, graph.getIterationSourceSinkPairs().size());
 
-		assertEquals(2, graph.getStreamLoops().size());
-		for (StreamLoop loop : graph.getStreamLoops()) {
-			assertEquals(loop.getHeads(), loop.getTails());
-			List<Tuple2<StreamNode, StreamNode>> sourceSinkPairs = loop.getSourceSinkPairs();
-			assertEquals(1, sourceSinkPairs.size());
+		for (Tuple2<StreamNode, StreamNode> sourceSinkPair: graph.getIterationSourceSinkPairs()) {
+			assertEquals(sourceSinkPair.f0.getOutEdges().get(0).getTargetVertex(), sourceSinkPair.f1.getInEdges().get(0).getSourceVertex());
 		}
 	}
 
 	@Test
 	public void testmultipleHeadsTailsSimple() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(4, MEMORYSIZE);
-		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5).shuffle();
-		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
+		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
+				.shuffle()
+				.map(NoOpIntMap).name("ParallelizeMapShuffle");
+		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
+				.map(NoOpIntMap).name("ParallelizeMapRebalance");
 
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap);
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap).setParallelism(2);
-		DataStream<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(2)
-				.addSink(new NoOpSink<Integer>());
-		DataStream<Integer> head4 = iter1.map(NoOpIntMap).addSink(new NoOpSink<Integer>());
+		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
 
-		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5).split(
-				new OutputSelector<Integer>() {
+		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2);
+		DataStream<Integer> head2 = iter1.map(NoOpIntMap).name("IterForwardMap");
+		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink<Integer>());
+		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
 
-					@Override
-					public Iterable<String> select(Integer value) {
-						return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd");
-					}
-				});
+		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
+				.map(NoOpIntMap).name("EvenOddSourceMap")
+				.split(new EvenOddOutputSelector());
 
 		iter1.closeWith(source3.select("even").union(
-				head1.map(NoOpIntMap).broadcast().setParallelism(1), head2.shuffle()));
+				head1.rebalance().map(NoOpIntMap).broadcast(), head2.shuffle()));
 
 		StreamGraph graph = env.getStreamGraph();
 
 		JobGraph jg = graph.getJobGraph();
 
-		assertEquals(1, graph.getStreamLoops().size());
-		StreamLoop loop = new ArrayList<StreamLoop>(graph.getStreamLoops()).get(0);
+		assertEquals(1, graph.getIterationSourceSinkPairs().size());
 
-		assertEquals(4, loop.getHeads().size());
-		assertEquals(3, loop.getTails().size());
+		Tuple2<StreamNode, StreamNode> sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next();
+		StreamNode itSource = sourceSinkPair.f0;
+		StreamNode itSink = sourceSinkPair.f1;
 
-		assertEquals(1, loop.getSourceSinkPairs().size());
-		Tuple2<StreamNode, StreamNode> pair = loop.getSourceSinkPairs().get(0);
+		assertEquals(4, itSource.getOutEdges().size());
+		assertEquals(3, itSink.getInEdges().size());
 
-		assertEquals(pair.f0.getParallelism(), pair.f1.getParallelism());
-		assertEquals(4, pair.f0.getOutEdges().size());
-		assertEquals(3, pair.f1.getInEdges().size());
+		assertEquals(itSource.getParallelism(), itSink.getParallelism());
 
-		for (StreamEdge edge : pair.f0.getOutEdges()) {
-			assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
-		}
-		for (StreamEdge edge : pair.f1.getInEdges()) {
-			assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
+		for (StreamEdge edge : itSource.getOutEdges()) {
+			if (edge.getTargetVertex().getOperatorName().equals("IterRebalanceMap")) {
+				assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
+			} else if (edge.getTargetVertex().getOperatorName().equals("IterForwardMap")) {
+				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
+			}
 		}
+		for (StreamEdge edge : itSink.getInEdges()) {
+			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapShuffle")) {
+				assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
+			}
 
-		assertTrue(loop.getTailSelectedNames().contains(Arrays.asList("even")));
+			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapForward")) {
+				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
+			}
+
+			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("EvenOddSourceMap")) {
+				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
+				assertTrue(edge.getSelectedNames().contains("even"));
+			}
+		}
 
 		// Test co-location
 
@@ -193,142 +277,106 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
 	public void testmultipleHeadsTailsWithTailPartitioning() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(4, MEMORYSIZE);
-		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5).shuffle();
-		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
+		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
+				.shuffle()
+				.map(NoOpIntMap);
 
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap);
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap).setParallelism(2).name("shuffle");
-		DataStream<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(2)
-				.addSink(new NoOpSink<Integer>());
-		DataStream<Integer> head4 = iter1.map(NoOpIntMap).addSink(new NoOpSink<Integer>());
+		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
+				.map(NoOpIntMap);
 
-		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5).name("split")
-				.split(new OutputSelector<Integer>() {
+		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
 
-					@Override
-					public Iterable<String> select(Integer value) {
-						return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd");
-					}
-				});
+		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("map1");
+		DataStream<Integer> head2 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).rebalance().name(
+				"shuffle");
+		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2)
+				.addSink(new ReceiveCheckNoOpSink<Integer>());
+		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
+
+		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
+				.map(NoOpIntMap)
+				.name("split")
+				.split(new EvenOddOutputSelector());
 
 		iter1.closeWith(
 				source3.select("even").union(
-						head1.map(NoOpIntMap).broadcast().setParallelism(1).name("bc"),
-						head2.shuffle()), true);
+						head1.map(NoOpIntMap).broadcast().name("bc"),
+						head2.map(NoOpIntMap).shuffle()));
 
 		StreamGraph graph = env.getStreamGraph();
 
 		JobGraph jg = graph.getJobGraph();
 
-		assertEquals(1, graph.getStreamLoops().size());
+		assertEquals(1, graph.getIterationSourceSinkPairs().size());
 
-		StreamLoop loop = new ArrayList<StreamLoop>(graph.getStreamLoops()).get(0);
+		Tuple2<StreamNode, StreamNode> sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next();
+		StreamNode itSource = sourceSinkPair.f0;
+		StreamNode itSink = sourceSinkPair.f1;
 
-		assertEquals(4, loop.getHeads().size());
-		assertEquals(3, loop.getTails().size());
+		assertEquals(4, itSource.getOutEdges().size());
+		assertEquals(3, itSink.getInEdges().size());
 
-		assertEquals(2, loop.getSourceSinkPairs().size());
-		List<Tuple2<StreamNode, StreamNode>> pairs = loop.getSourceSinkPairs();
-		Tuple2<StreamNode, StreamNode> pair1 = pairs.get(0).f0.getParallelism() == 2 ? pairs.get(0)
-				: pairs.get(1);
-		Tuple2<StreamNode, StreamNode> pair2 = pairs.get(0).f0.getParallelism() == 4 ? pairs.get(0)
-				: pairs.get(1);
 
-		assertEquals(pair1.f0.getParallelism(), pair1.f1.getParallelism());
-		assertEquals(2, pair1.f0.getParallelism());
-		assertEquals(2, pair1.f0.getOutEdges().size());
-		assertEquals(3, pair1.f1.getInEdges().size());
+		assertEquals(itSource.getParallelism(), itSink.getParallelism());
 
-		for (StreamEdge edge : pair1.f0.getOutEdges()) {
-			assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
-			assertEquals(2, edge.getTargetVertex().getParallelism());
-		}
-		for (StreamEdge edge : pair1.f1.getInEdges()) {
-			String tailName = edge.getSourceVertex().getOperatorName();
-			if (tailName.equals("split")) {
+		for (StreamEdge edge : itSource.getOutEdges()) {
+			if (edge.getTargetVertex().getOperatorName().equals("map1")) {
+				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
+				assertEquals(4, edge.getTargetVertex().getParallelism());
+			} else if (edge.getTargetVertex().getOperatorName().equals("shuffle")) {
 				assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
-			} else if (tailName.equals("bc")) {
-				assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner);
-			} else if (tailName.equals("shuffle")) {
-				assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
+				assertEquals(2, edge.getTargetVertex().getParallelism());
 			}
-
 		}
-
-		assertEquals(pair2.f0.getParallelism(), pair2.f1.getParallelism());
-		assertEquals(4, pair2.f0.getParallelism());
-		assertEquals(2, pair2.f0.getOutEdges().size());
-		assertEquals(3, pair2.f1.getInEdges().size());
-
-		for (StreamEdge edge : pair2.f0.getOutEdges()) {
-			assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
-			assertEquals(4, edge.getTargetVertex().getParallelism());
-		}
-		for (StreamEdge edge : pair2.f1.getInEdges()) {
+		for (StreamEdge edge : itSink.getInEdges()) {
 			String tailName = edge.getSourceVertex().getOperatorName();
 			if (tailName.equals("split")) {
-				assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
+				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
+				assertTrue(edge.getSelectedNames().contains("even"));
 			} else if (tailName.equals("bc")) {
 				assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner);
 			} else if (tailName.equals("shuffle")) {
 				assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
 			}
-
 		}
 
-		assertTrue(loop.getTailSelectedNames().contains(Arrays.asList("even")));
-
 		// Test co-location
 
 		JobVertex itSource1 = null;
-		JobVertex itSource2 = null;
 		JobVertex itSink1 = null;
-		JobVertex itSink2 = null;
 
 		for (JobVertex vertex : jg.getVertices()) {
 			if (vertex.getName().contains("IterationSource")) {
-				if (vertex.getName().contains("_0")) {
-					itSource1 = vertex;
-				} else if (vertex.getName().contains("_1")) {
-					itSource2 = vertex;
-				}
+				itSource1 = vertex;
 			} else if (vertex.getName().contains("IterationSink")) {
-				if (vertex.getName().contains("_0")) {
-					itSink1 = vertex;
-				} else if (vertex.getName().contains("_1")) {
-					itSink2 = vertex;
-				}
+				itSink1 = vertex;
 			}
 		}
 
 		assertTrue(itSource1.getCoLocationGroup() != null);
-		assertTrue(itSource2.getCoLocationGroup() != null);
+		assertTrue(itSink1.getCoLocationGroup() != null);
 
 		assertEquals(itSource1.getCoLocationGroup(), itSink1.getCoLocationGroup());
-		assertEquals(itSource2.getCoLocationGroup(), itSink2.getCoLocationGroup());
-		assertNotEquals(itSource1.getCoLocationGroup(), itSource2.getCoLocationGroup());
 	}
 
 	@SuppressWarnings("rawtypes")
 	@Test
 	public void testSimpleIteration() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-		iterated = new boolean[PARALLELISM];
+		iterated = new boolean[DEFAULT_PARALLELISM];
 
-		DataStream<Boolean> source = env
-				.fromCollection(Collections.nCopies(PARALLELISM * 2, false));
+		DataStream<Boolean> source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
+				.map(NoOpBoolMap).name("ParallelizeMap");
 
 		IterativeDataStream<Boolean> iteration = source.iterate(3000);
 
 		DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).map(NoOpBoolMap);
 
-		iteration.map(NoOpBoolMap).addSink(new NoOpSink());
+		iteration.map(NoOpBoolMap).addSink(new ReceiveCheckNoOpSink());
 
-		iteration.closeWith(increment).addSink(new NoOpSink());
+		iteration.closeWith(increment).addSink(new ReceiveCheckNoOpSink());
 
 		env.execute();
 
@@ -343,7 +391,13 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(2);
 
-		ConnectedIterativeDataStream<Integer, String> coIt = env.fromElements(0, 0).iterate(2000)
+		DataStream<String> otherSource = env.fromElements("1000", "2000")
+				.map(NoOpStrMap).name("ParallelizeMap");
+
+
+		ConnectedIterativeDataStream<Integer, String> coIt = env.fromElements(0, 0)
+				.map(NoOpIntMap).name("ParallelizeMap")
+				.iterate(2000)
 				.withFeedbackType("String");
 
 		try {
@@ -392,25 +446,24 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 			public String map2(String value) throws Exception {
 				return value;
 			}
-		}).setParallelism(1).addSink(new NoOpSink<String>());
+		}).addSink(new ReceiveCheckNoOpSink<String>());
 
-		coIt.closeWith(head.broadcast().union(env.fromElements("1000", "2000").rebalance()));
+		coIt.closeWith(head.broadcast().union(otherSource));
 
 		head.addSink(new TestSink()).setParallelism(1);
 
+		assertEquals(1, env.getStreamGraph().getIterationSourceSinkPairs().size());
+
 		env.execute();
 
 		Collections.sort(TestSink.collected);
 		assertEquals(Arrays.asList("1", "1", "2", "2", "2", "2"), TestSink.collected);
-		assertEquals(2, new ArrayList<StreamLoop>(env.getStreamGraph().getStreamLoops()).get(0)
-				.getSourceSinkPairs().size());
-
 	}
 
 	@Test
 	public void testGroupByFeedback() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
+		env.setParallelism(DEFAULT_PARALLELISM - 1);
 
 		KeySelector<Integer, Integer> key = new KeySelector<Integer, Integer>() {
 
@@ -420,7 +473,8 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 			}
 		};
 
-		DataStream<Integer> source = env.fromElements(1, 2, 3);
+		DataStream<Integer> source = env.fromElements(1, 2, 3)
+				.map(NoOpIntMap).name("ParallelizeMap");
 
 		IterativeDataStream<Integer> it = source.groupBy(key).iterate(3000);
 
@@ -448,8 +502,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		it.closeWith(head.groupBy(key).union(head.map(NoOpIntMap).setParallelism(2).groupBy(key)),
-				true).addSink(new NoOpSink<Integer>());
+		it.closeWith(head.groupBy(key).union(head.map(NoOpIntMap).groupBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
 
 		env.execute();
 	}
@@ -457,15 +510,17 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 	@SuppressWarnings("deprecation")
 	@Test
 	public void testWithCheckPointing() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
 		env.enableCheckpointing();
 
-		DataStream<Boolean> source = env
-				.fromCollection(Collections.nCopies(PARALLELISM * 2, false));
+		DataStream<Boolean> source = env .fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
+				.map(NoOpBoolMap).name("ParallelizeMap");
+
 
 		IterativeDataStream<Boolean> iteration = source.iterate(3000);
 
-		iteration.closeWith(iteration.flatMap(new IterationHead()));
+		iteration.closeWith(iteration.flatMap(new IterationHead())).addSink(new ReceiveCheckNoOpSink<Boolean>());
 
 		try {
 			env.execute();
@@ -503,22 +558,6 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	public static final class NoOpSink<T> extends RichSinkFunction<T> {
-		private List<T> received;
-
-		public void invoke(T tuple) {
-			received.add(tuple);
-		}
-
-		public void open(Configuration conf) {
-			received = new ArrayList<T>();
-		}
-
-		public void close() {
-			assertTrue(received.size() > 0);
-		}
-	}
-
 	public static CoMapFunction<Integer, String, String> NoOpCoMap = new CoMapFunction<Integer, String, String>() {
 
 		public String map1(Integer value) throws Exception {
@@ -530,9 +569,23 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		}
 	};
 
-	public static MapFunction<Integer, Integer> NoOpIntMap = new MapFunction<Integer, Integer>() {
+	public static MapFunction<Integer, Integer> NoOpIntMap = new NoOpIntMap();
+
+	public static MapFunction<String, String> NoOpStrMap = new MapFunction<String, String>() {
+
+		public String map(String value) throws Exception {
+			return value;
+		}
+
+	};
+
+	public static CoMapFunction<Integer, Integer, Integer> NoOpIntCoMap = new CoMapFunction<Integer, Integer, Integer>() {
+
+		public Integer map1(Integer value) throws Exception {
+			return value;
+		}
 
-		public Integer map(Integer value) throws Exception {
+		public Integer map2(Integer value) throws Exception {
 			return value;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
index fc78d27..8525d37 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
@@ -26,13 +26,12 @@ import java.util.List;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
 
-public class OutputSplitterTest {
-
-	private static final long MEMORYSIZE = 32;
+public class OutputSplitterTest extends StreamingMultipleProgramsTestBase {
 
 	private static ArrayList<Integer> expectedSplitterResult = new ArrayList<Integer>();
 
@@ -42,7 +41,8 @@ public class OutputSplitterTest {
 		TestListResultSink<Integer> splitterResultSink1 = new TestListResultSink<Integer>();
 		TestListResultSink<Integer> splitterResultSink2 = new TestListResultSink<Integer>();
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 		env.setBufferTimeout(1);
 
 		DataStream<Integer> d1 = env.fromElements(0, 2, 4, 6, 8);
@@ -95,7 +95,8 @@ public class OutputSplitterTest {
 		TestListResultSink<Integer> splitterResultSink1 = new TestListResultSink<Integer>();
 		TestListResultSink<Integer> splitterResultSink2 = new TestListResultSink<Integer>();
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 		env.setBufferTimeout(1);
 
 		DataStream<Integer> ds = env.fromElements(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
index c858834..987a8fb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
@@ -25,14 +25,18 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 
+import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.NoOpIntMap;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
@@ -40,10 +44,33 @@ import org.junit.Test;
 /**
  * IT case that tests the different stream partitioning schemes.
  */
-public class PartitionerTest {
+public class PartitionerTest extends StreamingMultipleProgramsTestBase {
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testForwardFailsLowToHighParallelism() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Integer> src = env.fromElements(1, 2, 3);
+
+		// this doesn't work because it goes from 1 to 3
+		src.forward().map(new NoOpIntMap());
+
+		env.execute();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testForwardFailsHightToLowParallelism() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// this does a rebalance that works
+		DataStream<Integer> src = env.fromElements(1, 2, 3).map(new NoOpIntMap());
+
+		// this doesn't work because it goes from 3 to 1
+		src.forward().map(new NoOpIntMap()).setParallelism(1);
+
+		env.execute();
+	}
 
-	public static final int PARALLELISM = 3;
-	public static final int MEMORY_SIZE = 32;
 
 	@Test
 	public void partitionerTest() {
@@ -62,7 +89,9 @@ public class PartitionerTest {
 				new TestListResultSink<Tuple2<Integer, String>>();
 
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
 		DataStream<Tuple1<String>> src = env.fromElements(
 				new Tuple1<String>("a"),
 				new Tuple1<String>("b"),
@@ -98,12 +127,21 @@ public class PartitionerTest {
 		// partition broadcast
 		src.broadcast().map(new SubtaskIndexAssigner()).addSink(broadcastPartitionResultSink);
 
-		// partition forward
-		src.map(new SubtaskIndexAssigner()).addSink(forwardPartitionResultSink);
-
 		// partition rebalance
 		src.rebalance().map(new SubtaskIndexAssigner()).addSink(rebalancePartitionResultSink);
 
+		// partition forward
+		src.map(new MapFunction<Tuple1<String>, Tuple1<String>>() {
+			private static final long serialVersionUID = 1L;
+			@Override
+			public Tuple1<String> map(Tuple1<String> value) throws Exception {
+				return value;
+			}
+		})
+				.forward()
+				.map(new SubtaskIndexAssigner())
+				.addSink(forwardPartitionResultSink);
+
 		// partition global
 		src.global().map(new SubtaskIndexAssigner()).addSink(globalPartitionResultSink);
 
@@ -209,8 +247,8 @@ public class PartitionerTest {
 				new HashSet<Tuple2<Integer, String>>(globalPartitionResult));
 	}
 
-	private static class SubtaskIndexAssigner
-			extends RichMapFunction<Tuple1<String>, Tuple2<Integer, String>> {
+	private static class SubtaskIndexAssigner extends RichMapFunction<Tuple1<String>, Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
 
 		private int indexOfSubtask;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
index e2fe599..606259e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
@@ -37,24 +37,23 @@ import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.util.NoOpSink;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.SplittableIterator;
 import org.junit.Test;
 
-public class StreamExecutionEnvironmentTest {
-
-	private static final long MEMORYSIZE = 32;
-	private static int PARALLELISM = 4;
+public class StreamExecutionEnvironmentTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
 	@SuppressWarnings("unchecked")
 	public void testFromCollectionParallelism() {
 		try {
 			TypeInformation<Integer> typeInfo = BasicTypeInfo.INT_TYPE_INFO;
-			StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 			DataStreamSource<Integer> dataStream1 = env.fromCollection(new DummySplittableIterator<Integer>(), typeInfo);
-			
+
 			try {
 				dataStream1.setParallelism(4);
 				fail("should throw an exception");
@@ -62,15 +61,20 @@ public class StreamExecutionEnvironmentTest {
 			catch (IllegalArgumentException e) {
 				// expected
 			}
+
+			dataStream1.addSink(new NoOpSink<Integer>());
 	
-			env.fromParallelCollection(new DummySplittableIterator<Integer>(), typeInfo).setParallelism(4);
-	
+			DataStreamSource<Integer> dataStream2 = env.fromParallelCollection(new DummySplittableIterator<Integer>(),
+					typeInfo).setParallelism(4);
+
+			dataStream2.addSink(new NoOpSink<Integer>());
+
 			String plan = env.getExecutionPlan();
-			
-			assertTrue("Parallelism for dataStream1 is not right.",
-					plan.contains("\"contents\":\"Collection Source\",\"parallelism\":1"));
-			assertTrue("Parallelism for dataStream2 is not right.",
-					plan.contains("\"contents\":\"Parallel Collection Source\",\"parallelism\":4"));
+
+			assertEquals("Parallelism of collection source must be 1.", 1, env.getStreamGraph().getStreamNode(dataStream1.getId()).getParallelism());
+			assertEquals("Parallelism of parallel collection source must be 4.",
+					4,
+					env.getStreamGraph().getStreamNode(dataStream2.getId()).getParallelism());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -80,7 +84,7 @@ public class StreamExecutionEnvironmentTest {
 
 	@Test
 	public void testSources() {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(PARALLELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		SourceFunction<Integer> srcFun = new SourceFunction<Integer>() {
 			private static final long serialVersionUID = 1L;
@@ -94,6 +98,7 @@ public class StreamExecutionEnvironmentTest {
 			}
 		};
 		DataStreamSource<Integer> src1 = env.addSource(srcFun);
+		src1.addSink(new NoOpSink<Integer>());
 		assertEquals(srcFun, getFunctionFromDataSource(src1));
 
 		List<Long> list = Arrays.asList(0L, 1L, 2L);
@@ -120,6 +125,7 @@ public class StreamExecutionEnvironmentTest {
 	}
 
 	private static <T> SourceFunction<T> getFunctionFromDataSource(DataStreamSource<T> dataStreamSource) {
+		dataStreamSource.addSink(new NoOpSink<T>());
 		AbstractUdfStreamOperator<?, ?> operator =
 				(AbstractUdfStreamOperator<?, ?>) getOperatorFromDataStream(dataStreamSource);
 		return (SourceFunction<T>) operator.getUserFunction();

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
index 7d08709..0989128 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
@@ -33,11 +33,12 @@ import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
 import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
 @SuppressWarnings("serial")
-public class TypeFillTest {
+public class TypeFillTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
 	public void test() {

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
index cccec40..508f1a2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
@@ -30,15 +30,12 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
 
-public class WindowCrossJoinTest implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final long MEMORYSIZE = 32;
+public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
 
 	private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> joinExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
 	private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> crossExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
@@ -55,7 +52,8 @@ public class WindowCrossJoinTest implements Serializable {
 
 	@Test
 	public void test() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 		env.setBufferTimeout(1);
 
 		TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>> joinResultSink =

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index fc3e36f..b7df2ec 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -30,11 +30,11 @@ import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
 
-public class DirectedOutputTest {
+public class DirectedOutputTest extends StreamingMultipleProgramsTestBase {
 
 	private static final String TEN = "ten";
 	private static final String ODD = "odd";
@@ -94,7 +94,8 @@ public class DirectedOutputTest {
 
 	@Test
 	public void outputSelectorTest() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
 
 		TestListResultSink<Long> evenSink = new TestListResultSink<Long>();
 		TestListResultSink<Long> oddAndTenSink = new TestListResultSink<Long>();

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
index 46e4327..f2c253c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
@@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.complex;
 
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -44,7 +43,6 @@ import org.apache.flink.streaming.api.windowing.helper.Delta;
 import org.apache.flink.streaming.api.windowing.helper.Time;
 import org.apache.flink.streaming.api.windowing.helper.Timestamp;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.After;
 import org.junit.Before;
@@ -131,7 +129,6 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 			}
 			
 		})
-				
 				.setParallelism(1).filter(new FilterFunction
 				<Tuple2<Long, Tuple2<String, Long>>>() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
index 767eaa4..39a13b3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
@@ -25,14 +25,15 @@ import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.junit.Test;
 
-public class SlotAllocationTest {
+public class SlotAllocationTest extends StreamingMultipleProgramsTestBase{
 
 	@SuppressWarnings("serial")
 	@Test
 	public void test() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(8);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		FilterFunction<Long> dummyFilter = new FilterFunction<Long>() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
new file mode 100644
index 0000000..fb2ef56
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.graph;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
+import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.util.EvenOddOutputSelector;
+import org.apache.flink.streaming.util.NoOpIntMap;
+import org.apache.flink.streaming.util.NoOpSink;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link StreamGraphGenerator}. This only tests correct translation of split/select,
+ * union, partitioning since the other translation routines are tested already in operation
+ * specific tests, for example in {@link org.apache.flink.streaming.api.IterateTest} for
+ * iterations.
+ */
+public class StreamGraphGeneratorTest extends StreamingMultipleProgramsTestBase {
+
+	/**
+	 * This tests whether virtual Transformations behave correctly.
+	 *
+	 * <p>
+	 * Verifies that partitioning, output selector, selected names are correctly set in the
+	 * StreamGraph when they are intermixed.
+	 */
+	@Test
+	public void testVirtualTransformations() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Integer> source = env.fromElements(1, 10);
+
+		DataStream<Integer> rebalanceMap = source.rebalance().map(new NoOpIntMap());
+
+		// verify that only the partitioning that was set last is used
+		DataStream<Integer> broadcastMap = rebalanceMap
+				.forward()
+				.global()
+				.broadcast()
+				.map(new NoOpIntMap());
+
+		broadcastMap.addSink(new NoOpSink<Integer>());
+
+		// verify that partitioning is preserved across union and split/select
+		EvenOddOutputSelector selector1 = new EvenOddOutputSelector();
+		EvenOddOutputSelector selector2 = new EvenOddOutputSelector();
+		EvenOddOutputSelector selector3 = new EvenOddOutputSelector();
+
+		DataStream<Integer> map1Operator = rebalanceMap
+				.map(new NoOpIntMap());
+
+		DataStream<Integer> map1 = map1Operator
+				.broadcast()
+				.split(selector1)
+				.select("even");
+
+		DataStream<Integer> map2Operator = rebalanceMap
+				.map(new NoOpIntMap());
+
+		DataStream<Integer> map2 = map2Operator
+				.split(selector2)
+				.select("odd")
+				.global();
+
+		DataStream<Integer> map3Operator = rebalanceMap
+				.map(new NoOpIntMap());
+
+		DataStream<Integer> map3 = map3Operator
+				.global()
+				.split(selector3)
+				.select("even")
+				.shuffle();
+
+
+		SingleOutputStreamOperator<Integer, ?> unionedMap = map1.union(map2).union(map3)
+				.map(new NoOpIntMap());
+
+		unionedMap.addSink(new NoOpSink<Integer>());
+
+		StreamGraph graph = env.getStreamGraph();
+
+		// rebalanceMap
+		assertTrue(graph.getStreamNode(rebalanceMap.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner);
+
+		// verify that only last partitioning takes precedence
+		assertTrue(graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
+		assertEquals(rebalanceMap.getId(), graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getSourceVertex().getId());
+
+		// verify that partitioning in unions is preserved and that it works across split/select
+		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
+		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even"));
+		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutputSelectors().contains(selector1));
+
+		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof GlobalPartitioner);
+		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("odd"));
+		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutputSelectors().contains(selector2));
+
+		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof ShufflePartitioner);
+		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even"));
+		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutputSelectors().contains(selector3));
+	}
+
+	/**
+	 * This tests whether virtual Transformations behave correctly.
+	 *
+	 * Checks whether output selector, partitioning works correctly when applied on a union.
+	 */
+	@Test
+	public void testVirtualTransformations2() throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Integer> source = env.fromElements(1, 10);
+
+		DataStream<Integer> rebalanceMap = source.rebalance().map(new NoOpIntMap());
+
+		DataStream<Integer> map1 = rebalanceMap
+				.map(new NoOpIntMap());
+
+		DataStream<Integer> map2 = rebalanceMap
+				.map(new NoOpIntMap());
+
+		DataStream<Integer> map3 = rebalanceMap
+				.map(new NoOpIntMap());
+
+		EvenOddOutputSelector selector = new EvenOddOutputSelector();
+
+		SingleOutputStreamOperator<Integer, ?> unionedMap = map1.union(map2).union(map3)
+				.broadcast()
+				.split(selector)
+				.select("foo")
+				.map(new NoOpIntMap());
+
+		unionedMap.addSink(new NoOpSink<Integer>());
+
+		StreamGraph graph = env.getStreamGraph();
+
+		// verify that the properties are correctly set on all input operators
+		assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
+		assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
+		assertTrue(graph.getStreamNode(map1.getId()).getOutputSelectors().contains(selector));
+
+		assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
+		assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
+		assertTrue(graph.getStreamNode(map2.getId()).getOutputSelectors().contains(selector));
+
+		assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
+		assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
+		assertTrue(graph.getStreamNode(map3.getId()).getOutputSelectors().contains(selector));
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
index 4e7c963..e806428 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
@@ -21,6 +21,8 @@ import java.util.Random;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.InstantiationUtil;
 import org.junit.Assert;
@@ -28,7 +30,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class StreamingJobGraphGeneratorTest {
+public class StreamingJobGraphGeneratorTest extends StreamingMultipleProgramsTestBase {
 	private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGeneratorTest.class);
 	
 	@Test
@@ -36,8 +38,9 @@ public class StreamingJobGraphGeneratorTest {
 		final long seed = System.currentTimeMillis();
 		LOG.info("Test seed: {}", new Long(seed));
 		final Random r = new Random(seed);
-		
-		TestStreamEnvironment env = new TestStreamEnvironment(4, 32);
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
 		StreamGraph streamingJob = new StreamGraph(env);
 		StreamingJobGraphGenerator compiler = new StreamingJobGraphGenerator(streamingJob);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
index e8f0a03..bb9dad7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
@@ -40,6 +40,7 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestHarnessUtil;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
@@ -52,8 +53,7 @@ import org.junit.Test;
  *     <li>Watermarks are correctly forwarded</li>
  * </ul>
  */
-public class StreamProjectTest implements Serializable {
-	private static final long serialVersionUID = 1L;
+public class StreamProjectTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
 	public void testProject() throws Exception {
@@ -95,7 +95,6 @@ public class StreamProjectTest implements Serializable {
 
 
 	// tests using projection from the API without explicitly specifying the types
-	private static final long MEMORY_SIZE = 32;
 	private static HashSet<Tuple2<Long, Double>> expected = new HashSet<Tuple2<Long, Double>>();
 	private static HashSet<Tuple2<Long, Double>> actual = new HashSet<Tuple2<Long, Double>>();
 
@@ -106,7 +105,8 @@ public class StreamProjectTest implements Serializable {
 			expected.add(new Tuple2<Long, Double>(i, i.doubleValue()));
 		}
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 
 		env.generateSequence(1, 10).map(new MapFunction<Long, Tuple3<Long, Character, Double>>() {
 			private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
index b8e9619..4c644a9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
@@ -30,10 +30,12 @@ import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.NoOpSink;
 import org.apache.flink.streaming.util.TestHarnessUtil;
 import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -169,21 +171,4 @@ public class CoStreamFlatMapTest implements Serializable {
 		}
 
 	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void multipleInputTest() {
-		LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		DataStream<Integer> ds1 = env.fromElements(1, 3, 5);
-		DataStream<Integer> ds2 = env.fromElements(2, 4).union(ds1);
-		
-		try {
-			ds1.forward().union(ds2);
-			fail();
-		} catch (RuntimeException e) {
-			// expected
-		}
-		
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
index 5986a30..512a0df 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
@@ -31,19 +31,17 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
-public class SelfConnectionTest implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final int MEMORY_SIZE = 32;
+public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
 
 	private static List<String> expected;
 
@@ -51,20 +49,12 @@ public class SelfConnectionTest implements Serializable {
 	@Test
 	public void sameDataStreamTest() {
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
 
 		TestListResultSink<String> resultSink = new TestListResultSink<String>();
 
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
+		Timestamp<Integer> timeStamp = new IntegerTimestamp();
 
 		KeySelector keySelector = new KeySelector<Integer, Integer>() {
 
@@ -79,7 +69,7 @@ public class SelfConnectionTest implements Serializable {
 		DataStream<Integer> src = env.fromElements(1, 3, 5);
 
 		@SuppressWarnings("unused")
-		DataStream<Tuple2<Integer, Integer>> dataStream =
+		DataStreamSink<Tuple2<Integer, Integer>> dataStream =
 				src.join(src).onWindow(50L, timeStamp, timeStamp).where(keySelector).equalTo(keySelector)
 						.map(new MapFunction<Tuple2<Integer, Integer>, String>() {
 
@@ -107,8 +97,8 @@ public class SelfConnectionTest implements Serializable {
 
 			assertEquals(expected, result);
 		} catch (Exception e) {
-			fail();
 			e.printStackTrace();
+			fail();
 		}
 	}
 
@@ -120,7 +110,8 @@ public class SelfConnectionTest implements Serializable {
 
 		TestListResultSink<String> resultSink = new TestListResultSink<String>();
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(1, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 
 		DataStream<Integer> src = env.fromElements(1, 3, 5);
 
@@ -175,7 +166,8 @@ public class SelfConnectionTest implements Serializable {
 
 		TestListResultSink<String> resultSink = new TestListResultSink<String>();
 
-		StreamExecutionEnvironment env = new TestStreamEnvironment(3, MEMORY_SIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
 
 		DataStream<Integer> src = env.fromElements(1, 3, 5).disableChaining();
 
@@ -248,4 +240,15 @@ public class SelfConnectionTest implements Serializable {
 
 		assertEquals(expected, result);
 	}
+
+	private static class IntegerTimestamp implements Timestamp<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Integer value) {
+			return value;
+		}
+
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
index 5e6ffa2..db09373 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
@@ -40,14 +40,12 @@ import org.apache.flink.streaming.api.windowing.helper.Count;
 import org.apache.flink.streaming.api.windowing.helper.FullStream;
 import org.apache.flink.streaming.api.windowing.helper.Time;
 import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
-public class WindowingITCase implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	private static final Integer MEMORYSIZE = 32;
+public class WindowingITCase extends StreamingMultipleProgramsTestBase {
 
 	@SuppressWarnings("serial")
 	public static class ModKey implements KeySelector<Integer, Integer> {
@@ -98,17 +96,10 @@ public class WindowingITCase implements Serializable {
 
 		KeySelector<Integer, ?> key = new ModKey(2);
 
-		Timestamp<Integer> ts = new Timestamp<Integer>() {
-
-			private static final long serialVersionUID = 1L;
+		Timestamp<Integer> ts = new IntegerTimestamp();
 
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-		};
-
-		StreamExecutionEnvironment env = new TestStreamEnvironment(2, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(2);
 		env.disableOperatorChaining();
 
 		DataStream<Integer> source = env.fromCollection(inputs);
@@ -116,14 +107,18 @@ public class WindowingITCase implements Serializable {
 		source.window(Time.of(3, ts, 1)).every(Time.of(2, ts, 1)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink1());
 
-		source.window(Time.of(4, ts, 1)).groupBy(new ModKey(2)).mapWindow(new IdentityWindowMap())
-				.flatten().addSink(new TestSink2());
+		source.window(Time.of(4, ts, 1)).groupBy(new ModKey(2))
+				.mapWindow(new IdentityWindowMap())
+				.flatten()
+				.addSink(new TestSink2()).name("TESTSIUNK2");
 
 		source.groupBy(key).window(Time.of(4, ts, 1)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink4());
 
 		source.groupBy(new ModKey(3)).window(Count.of(2)).groupBy(new ModKey(2))
-				.mapWindow(new IdentityWindowMap()).flatten().addSink(new TestSink5());
+				.mapWindow(new IdentityWindowMap())
+				.flatten()
+				.addSink(new TestSink5());
 
 		source.window(Time.of(2, ts)).every(Time.of(3, ts)).min(0).getDiscretizedStream()
 				.addSink(new TestSink3());
@@ -131,11 +126,13 @@ public class WindowingITCase implements Serializable {
 		source.groupBy(key).window(Time.of(4, ts, 1)).max(0).getDiscretizedStream()
 				.addSink(new TestSink6());
 
-		source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap()).flatten()
+		source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap())
+				.flatten()
 				.addSink(new TestSink7());
 
 		source.window(Time.of(5, ts, 1)).every(Time.of(4, ts, 1)).groupBy(new ModKey(2)).sum(0)
-				.getDiscretizedStream().addSink(new TestSink8());
+				.getDiscretizedStream()
+				.addSink(new TestSink8());
 
 		try {
 			source.window(FullStream.window()).every(Count.of(2)).getDiscretizedStream();
@@ -156,7 +153,8 @@ public class WindowingITCase implements Serializable {
 		source.every(Count.of(4)).sum(0).getDiscretizedStream().addSink(new TestSink11());
 
 		source.window(FullStream.window()).every(Count.of(4)).groupBy(key).sum(0)
-				.getDiscretizedStream().addSink(new TestSink12());
+				.getDiscretizedStream()
+				.addSink(new TestSink12());
 
 		DataStream<Integer> source2 = env.addSource(new ParallelSourceFunction<Integer>() {
 			private static final long serialVersionUID = 1L;
@@ -202,12 +200,15 @@ public class WindowingITCase implements Serializable {
 		source3.window(Time.of(5, ts, 1)).groupBy(new ModKey(2)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink10());
 
-		source.map(new MapFunction<Integer, Integer>() {
-			@Override
-			public Integer map(Integer value) throws Exception {
-				return value;
-			}
-		}).every(Time.of(5, ts, 1)).sum(0).getDiscretizedStream().addSink(new TestSink13());
+		source
+				.map(new MapFunction<Integer, Integer>() {
+					@Override
+					public Integer map(Integer value) throws Exception {
+						return value;
+					}
+				})
+				.every(Time.of(5, ts, 1)).sum(0).getDiscretizedStream()
+				.addSink(new TestSink13());
 
 		env.execute();
 
@@ -516,4 +517,13 @@ public class WindowingITCase implements Serializable {
 
 	}
 
+	private static class IntegerTimestamp implements Timestamp<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Integer value) {
+			return value;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
index 68e2a75..7ac5616 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
@@ -40,9 +40,9 @@ public class CsvOutputFormatITCase extends StreamingProgramTestBase {
 
 		DataStream<String> text = env.fromElements(WordCountData.TEXT);
 
-		DataStream<Tuple2<String, Integer>> counts =
-				text.flatMap(new Tokenizer())
-						.groupBy(0).sum(1);
+		DataStream<Tuple2<String, Integer>> counts = text
+				.flatMap(new Tokenizer())
+				.groupBy(0).sum(1);
 
 		counts.writeAsCsv(resultPath);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
index 3c48b3f..6bbcea8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
@@ -38,9 +38,9 @@ public class TextOutputFormatITCase extends StreamingProgramTestBase {
 
 		DataStream<String> text = env.fromElements(WordCountData.TEXT);
 
-		DataStream<Tuple2<String, Integer>> counts =
-				text.flatMap(new CsvOutputFormatITCase.Tokenizer())
-						.groupBy(0).sum(1);
+		DataStream<Tuple2<String, Integer>> counts = text
+				.flatMap(new CsvOutputFormatITCase.Tokenizer())
+				.groupBy(0).sum(1);
 
 		counts.writeAsText(resultPath);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
index 60db798..32b3455 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
@@ -51,6 +51,7 @@ import org.apache.flink.streaming.api.operators.StreamMap;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.InstantiationUtil;
 import org.junit.Test;
@@ -62,7 +63,7 @@ import com.google.common.collect.ImmutableMap;
  * partitioned and non-partitioned user states. This test mimics the runtime
  * behavior of stateful stream operators.
  */
-public class StatefulOperatorTest {
+public class StatefulOperatorTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
 	public void simpleStateTest() throws Exception {
@@ -104,7 +105,8 @@ public class StatefulOperatorTest {
 	
 	@Test
 	public void apiTest() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(3, 32);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
 
 		KeyedDataStream<Integer> keyedStream = env.fromCollection(Arrays.asList(0, 1, 2, 3, 4, 5, 6)).keyBy(new ModKey(4));
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
index f45125b..122aa8a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
@@ -36,10 +36,11 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.junit.Test;
 
-public class StreamVertexTest {
+public class StreamVertexTest extends StreamingMultipleProgramsTestBase {
 
 	private static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
 
@@ -86,14 +87,12 @@ public class StreamVertexTest {
 	}
 
 	@SuppressWarnings("unused")
-	private static final int PARALLELISM = 1;
 	private static final int SOURCE_PARALELISM = 1;
-	private static final long MEMORYSIZE = 32;
 
 	@Test
 	public void wrongJobGraph() {
-		LocalStreamEnvironment env = StreamExecutionEnvironment
-				.createLocalEnvironment(SOURCE_PARALELISM);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(SOURCE_PARALELISM);
 
 		try {
 			env.fromCollection(null);
@@ -155,7 +154,8 @@ public class StreamVertexTest {
 
 	@Test
 	public void coTest() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(SOURCE_PARALELISM);
 
 		DataStream<String> fromStringElements = env.fromElements("aa", "bb", "cc");
 		DataStream<Long> generatedSequence = env.generateSequence(0, 3);
@@ -171,7 +171,8 @@ public class StreamVertexTest {
 
 	@Test
 	public void runStream() throws Exception {
-		StreamExecutionEnvironment env = new TestStreamEnvironment(SOURCE_PARALELISM, MEMORYSIZE);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(SOURCE_PARALELISM);
 
 		env.addSource(new MySource()).setParallelism(SOURCE_PARALELISM).map(new MyTask())
 				.addSink(new MySink());

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
index 89679ea..9d9d47b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
@@ -24,12 +24,13 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
 @SuppressWarnings("serial")
-public class TranslationTest {
+public class TranslationTest extends StreamingMultipleProgramsTestBase {
 	
 	@Test
 	public void testCheckpointModeTranslation() {
@@ -66,7 +67,8 @@ public class TranslationTest {
 		env.generateSequence(1, 10000000)
 				.addSink(new SinkFunction<Long>() {
 					@Override
-					public void invoke(Long value) {}
+					public void invoke(Long value) {
+					}
 				});
 		
 		return env;

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java
deleted file mode 100644
index 2643bba..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/DistributePartitionerTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DistributePartitionerTest {
-	
-	private RebalancePartitioner<Tuple> distributePartitioner;
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
-			null);
-	
-	@Before
-	public void setPartitioner() {
-		distributePartitioner = new RebalancePartitioner<Tuple>(false);
-	}
-	
-	@Test
-	public void testSelectChannelsLength() {
-		sd.setInstance(streamRecord);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 1).length);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 2).length);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 1024).length);
-	}
-	
-	@Test
-	public void testSelectChannelsInterval() {
-		sd.setInstance(streamRecord);
-		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(2, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
-	}
-}


[4/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index 65736f5..6474ae9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -23,14 +23,15 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -44,11 +45,11 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.state.StateHandleProvider;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamSource;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
@@ -84,11 +85,15 @@ public class StreamGraph extends StreamingPlan {
 
 	private Map<Integer, StreamNode> streamNodes;
 	private Set<Integer> sources;
+	private Set<Integer> sinks;
+	private Map<Integer, Tuple2<Integer, List<String>>> virtualSelectNodes;
+	private Map<Integer, Tuple2<Integer, StreamPartitioner<?>>> virtuaPartitionNodes;
 
-	private Map<Integer, StreamLoop> streamLoops;
-	protected Map<Integer, StreamLoop> vertexIDtoLoop;
 	protected Map<Integer, String> vertexIDtoBrokerID;
+	protected Map<Integer, Long> vertexIDtoLoopTimeout;
 	private StateHandleProvider<?> stateHandleProvider;
+	private Set<Tuple2<StreamNode, StreamNode>> iterationSourceSinkPairs;
+
 	private boolean forceCheckpoint = false;
 
 	public StreamGraph(StreamExecutionEnvironment environment) {
@@ -104,11 +109,14 @@ public class StreamGraph extends StreamingPlan {
 	 * Remove all registered nodes etc.
 	 */
 	public void clear() {
-		streamNodes = new HashMap<Integer, StreamNode>();
-		streamLoops = new HashMap<Integer, StreamLoop>();
-		vertexIDtoLoop = new HashMap<Integer, StreamLoop>();
-		vertexIDtoBrokerID = new HashMap<Integer, String>();
-		sources = new HashSet<Integer>();
+		streamNodes = Maps.newHashMap();
+		virtualSelectNodes = Maps.newHashMap();
+		virtuaPartitionNodes = Maps.newHashMap();
+		vertexIDtoBrokerID = Maps.newHashMap();
+		vertexIDtoLoopTimeout = Maps.newHashMap();
+		iterationSourceSinkPairs = Sets.newHashSet();
+		sources = Sets.newHashSet();
+		sinks = Sets.newHashSet();
 	}
 
 	protected ExecutionConfig getExecutionConfig() {
@@ -167,7 +175,7 @@ public class StreamGraph extends StreamingPlan {
 	
 
 	public boolean isIterative() {
-		return !streamLoops.isEmpty();
+		return!vertexIDtoLoopTimeout.isEmpty();
 	}
 
 	public <IN, OUT> void addSource(Integer vertexID, StreamOperator<OUT> operatorObject,
@@ -176,6 +184,12 @@ public class StreamGraph extends StreamingPlan {
 		sources.add(vertexID);
 	}
 
+	public <IN, OUT> void addSink(Integer vertexID, StreamOperator<OUT> operatorObject,
+			TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
+		addOperator(vertexID, operatorObject, inTypeInfo, outTypeInfo, operatorName);
+		sinks.add(vertexID);
+	}
+
 	public <IN, OUT> void addOperator(Integer vertexID, StreamOperator<OUT> operatorObject,
 			TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
 
@@ -212,224 +226,141 @@ public class StreamGraph extends StreamingPlan {
 		}
 	}
 
-	public void addIterationHead(Integer iterationHead, Integer iterationID, long timeOut,
-			TypeInformation<?> feedbackType) {
-		// If there is no loop object created for this iteration create one
-		StreamLoop loop = streamLoops.get(iterationID);
-		if (loop == null) {
-			loop = new StreamLoop(iterationID, timeOut, feedbackType);
-			streamLoops.put(iterationID, loop);
-		}
-
-		loop.addHeadOperator(getStreamNode(iterationHead));
-	}
-
-	public void addIterationTail(List<DataStream<?>> feedbackStreams, Integer iterationID,
-			boolean keepPartitioning) {
+	protected StreamNode addNode(Integer vertexID, Class<? extends AbstractInvokable> vertexClass,
+			StreamOperator<?> operatorObject, String operatorName) {
 
-		if (!streamLoops.containsKey(iterationID)) {
-			throw new RuntimeException("Cannot close iteration without head operator.");
+		if (streamNodes.containsKey(vertexID)) {
+			throw new RuntimeException("Duplicate vertexID " + vertexID);
 		}
 
-		StreamLoop loop = streamLoops.get(iterationID);
+		StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName,
+				new ArrayList<OutputSelector<?>>(), vertexClass);
 
-		for (DataStream<?> stream : feedbackStreams) {
-			loop.addTailOperator(getStreamNode(stream.getId()), stream.getPartitioner(),
-					stream.getSelectedNames());
-		}
+		streamNodes.put(vertexID, vertex);
 
-		if (keepPartitioning) {
-			loop.applyTailPartitioning();
-		}
+		return vertex;
 	}
 
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public void finalizeLoops() {
-		
-		// We create each loop separately, the order does not matter as sinks
-		// and sources don't interact
-		for (StreamLoop loop : streamLoops.values()) {
-
-			// We make sure not to re-create the loops if the method is called
-			// multiple times
-			if (loop.getSourceSinkPairs().isEmpty()) {
-
-				List<StreamNode> headOps = loop.getHeads();
-				List<StreamNode> tailOps = loop.getTails();
+	/**
+	 * Adds a new virtual node that is used to connect a downstream vertex to only the outputs
+	 * with the selected names.
+	 *
+	 * When adding an edge from the virtual node to a downstream node the connection will be made
+	 * to the original node, only with the selected names given here.
+	 *
+	 * @param originalId ID of the node that should be connected to.
+	 * @param virtualId ID of the virtual node.
+	 * @param selectedNames The selected names.
+	 */
+	public void addVirtualSelectNode(Integer originalId, Integer virtualId, List<String> selectedNames) {
 
-				// This means that the iteration was not closed. It should not
-				// be
-				// allowed.
-				if (tailOps.isEmpty()) {
-					throw new RuntimeException("Cannot execute job with empty iterations.");
-				}
+		if (virtualSelectNodes.containsKey(virtualId)) {
+			throw new IllegalStateException("Already has virtual select node with id " + virtualId);
+		}
 
-				// Check whether we keep the feedback partitioning
-				if (loop.keepsPartitioning()) {
-					// This is the complicated case as we need to enforce
-					// partitioning on the tail -> sink side, which
-					// requires strict forward connections at source -> head
-
-					// We need one source/sink pair per different head
-					// parallelism
-					// as we depend on strict forwards connections
-					Map<Integer, List<StreamNode>> parallelismToHeads = new HashMap<Integer, List<StreamNode>>();
-
-					// Group head operators by parallelism
-					for (StreamNode head : headOps) {
-						int p = head.getParallelism();
-						if (!parallelismToHeads.containsKey(p)) {
-							parallelismToHeads.put(p, new ArrayList<StreamNode>());
-						}
-						parallelismToHeads.get(p).add(head);
-					}
-
-					// We create the sink/source pair for each parallelism
-					// group,
-					// tails will forward to all sinks but each head operator
-					// will
-					// only receive from one source (corresponding to its
-					// parallelism)
-					int c = 0;
-					for (Entry<Integer, List<StreamNode>> headGroup : parallelismToHeads.entrySet()) {
-						List<StreamNode> headOpsInGroup = headGroup.getValue();
-
-						Tuple2<StreamNode, StreamNode> sourceSinkPair = createItSourceAndSink(loop,
-								c);
-						StreamNode source = sourceSinkPair.f0;
-						StreamNode sink = sourceSinkPair.f1;
-
-						// We connect the source to the heads in this group
-						// (forward), setting
-						// type to 2 in case we have a coIteration (this sets
-						// the
-						// input as the second input of the co-operator)
-						for (StreamNode head : headOpsInGroup) {
-							int inputType = loop.isCoIteration() ? 2 : 0;
-							addEdge(source.getId(), head.getId(), new RebalancePartitioner(true),
-									inputType, new ArrayList<String>());
-						}
-
-						// We connect all the tails to the sink keeping the
-						// partitioner
-						for (int i = 0; i < tailOps.size(); i++) {
-							StreamNode tail = tailOps.get(i);
-							StreamPartitioner<?> partitioner = loop.getTailPartitioners().get(i);
-							addEdge(tail.getId(), sink.getId(), partitioner.copy(), 0, loop
-									.getTailSelectedNames().get(i));
-						}
-
-						// We set the sink/source parallelism to the group
-						// parallelism
-						source.setParallelism(headGroup.getKey());
-						sink.setParallelism(source.getParallelism());
-
-						// We set the proper serializers for the sink/source
-						setSerializersFrom(tailOps.get(0).getId(), sink.getId());
-						if (loop.isCoIteration()) {
-							source.setSerializerOut(loop.getFeedbackType().createSerializer(executionConfig));
-						} else {
-							setSerializersFrom(headOpsInGroup.get(0).getId(), source.getId());
-						}
-
-						c++;
-					}
-
-				} else {
-					// This is the most simple case, we add one iteration
-					// sink/source pair with the parallelism of the first tail
-					// operator. Tail operators will forward the records and
-					// partitioning will be enforced from source -> head
-
-					Tuple2<StreamNode, StreamNode> sourceSinkPair = createItSourceAndSink(loop, 0);
-					StreamNode source = sourceSinkPair.f0;
-					StreamNode sink = sourceSinkPair.f1;
-
-					// We get the feedback partitioner from the first input of
-					// the
-					// first head.
-					StreamPartitioner<?> partitioner = headOps.get(0).getInEdges().get(0)
-							.getPartitioner();
-
-					// Connect the sources to heads using this partitioner
-					for (StreamNode head : headOps) {
-						addEdge(source.getId(), head.getId(), partitioner.copy(), 0,
-								new ArrayList<String>());
-					}
-
-					// The tails are connected to the sink with forward
-					// partitioning
-					for (int i = 0; i < tailOps.size(); i++) {
-						StreamNode tail = tailOps.get(i);
-						addEdge(tail.getId(), sink.getId(), new RebalancePartitioner(true), 0, loop
-								.getTailSelectedNames().get(i));
-					}
-
-					// We set the parallelism to match the first tail op to make
-					// the
-					// forward more efficient
-					sink.setParallelism(tailOps.get(0).getParallelism());
-					source.setParallelism(sink.getParallelism());
-
-					// We set the proper serializers
-					setSerializersFrom(headOps.get(0).getId(), source.getId());
-					setSerializersFrom(tailOps.get(0).getId(), sink.getId());
-				}
+		virtualSelectNodes.put(virtualId,
+				new Tuple2<Integer, List<String>>(originalId, selectedNames));
+	}
 
-			}
+	/**
+	 * Adds a new virtual node that is used to connect a downstream vertex to an input with a certain
+	 * partitioning.
+	 *
+	 * When adding an edge from the virtual node to a downstream node the connection will be made
+	 * to the original node, but with the partitioning given here.
+	 *
+	 * @param originalId ID of the node that should be connected to.
+	 * @param virtualId ID of the virtual node.
+	 * @param partitioner The partitioner
+	 */
+	public void addVirtualPartitionNode(Integer originalId, Integer virtualId, StreamPartitioner<?> partitioner) {
 
+		if (virtuaPartitionNodes.containsKey(virtualId)) {
+			throw new IllegalStateException("Already has virtual partition node with id " + virtualId);
 		}
 
+		virtuaPartitionNodes.put(virtualId,
+				new Tuple2<Integer, StreamPartitioner<?>>(originalId, partitioner));
 	}
 
-	private Tuple2<StreamNode, StreamNode> createItSourceAndSink(StreamLoop loop, int c) {
-		StreamNode source = addNode(-1 * streamNodes.size(), StreamIterationHead.class, null, null);
-		sources.add(source.getId());
-
-		StreamNode sink = addNode(-1 * streamNodes.size(), StreamIterationTail.class, null, null);
-
-		source.setOperatorName("IterationSource-" + loop.getID() + "_" + c);
-		sink.setOperatorName("IterationSink-" + loop.getID() + "_" + c);
-		vertexIDtoBrokerID.put(source.getId(), loop.getID() + "_" + c);
-		vertexIDtoBrokerID.put(sink.getId(), loop.getID() + "_" + c);
-		vertexIDtoLoop.put(source.getId(), loop);
-		vertexIDtoLoop.put(sink.getId(), loop);
-		loop.addSourceSinkPair(source, sink);
+	public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, int typeNumber) {
+		addEdgeInternal(upStreamVertexID,
+				downStreamVertexID,
+				typeNumber,
+				null,
+				Lists.<String>newArrayList());
 
-		return new Tuple2<StreamNode, StreamNode>(source, sink);
 	}
 
-	protected StreamNode addNode(Integer vertexID, Class<? extends AbstractInvokable> vertexClass,
-			StreamOperator<?> operatorObject, String operatorName) {
+	private void addEdgeInternal(Integer upStreamVertexID,
+			Integer downStreamVertexID,
+			int typeNumber,
+			StreamPartitioner<?> partitioner,
+			List<String> outputNames) {
 
-		StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName,
-				new ArrayList<OutputSelector<?>>(), vertexClass);
 
-		streamNodes.put(vertexID, vertex);
+		if (virtualSelectNodes.containsKey(upStreamVertexID)) {
+			int virtualId = upStreamVertexID;
+			upStreamVertexID = virtualSelectNodes.get(virtualId).f0;
+			if (outputNames.isEmpty()) {
+				// selections that happen downstream override earlier selections
+				outputNames = virtualSelectNodes.get(virtualId).f1;
+			}
+			addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames);
+		} else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) {
+			int virtualId = upStreamVertexID;
+			upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0;
+			if (partitioner == null) {
+				partitioner = virtuaPartitionNodes.get(virtualId).f1;
+			}
+			addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames);
+		} else {
+			StreamNode upstreamNode = getStreamNode(upStreamVertexID);
+			StreamNode downstreamNode = getStreamNode(downStreamVertexID);
+
+			// If no partitioner was specified and the parallelism of upstream and downstream
+			// operator matches use forward partitioning, use rebalance otherwise.
+			if (partitioner == null && upstreamNode.getParallelism() == downstreamNode.getParallelism()) {
+				partitioner = new ForwardPartitioner<Object>();
+			} else if (partitioner == null) {
+				partitioner = new RebalancePartitioner<Object>();
+			}
 
-		return vertex;
-	}
+			if (partitioner instanceof ForwardPartitioner) {
+				if (upstreamNode.getParallelism() != downstreamNode.getParallelism()) {
+					throw new UnsupportedOperationException("Forward partitioning does not allow " +
+							"change of parallelism. Upstream operation: " + upstreamNode + " parallelism: " + upstreamNode.getParallelism() +
+							", downstream operation: " + downstreamNode + " parallelism: " + downstreamNode.getParallelism() +
+							" You must use another partitioning strategy, such as broadcast, rebalance, shuffle or global.");
+				}
+			}
 
-	public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID,
-			StreamPartitioner<?> partitionerObject, int typeNumber, List<String> outputNames) {
+			StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner);
 
-		StreamEdge edge = new StreamEdge(getStreamNode(upStreamVertexID),
-				getStreamNode(downStreamVertexID), typeNumber, outputNames, partitionerObject);
-		getStreamNode(edge.getSourceId()).addOutEdge(edge);
-		getStreamNode(edge.getTargetId()).addInEdge(edge);
+			getStreamNode(edge.getSourceId()).addOutEdge(edge);
+			getStreamNode(edge.getTargetId()).addInEdge(edge);
+		}
 	}
 
 	public <T> void addOutputSelector(Integer vertexID, OutputSelector<T> outputSelector) {
-		getStreamNode(vertexID).addOutputSelector(outputSelector);
+		if (virtuaPartitionNodes.containsKey(vertexID)) {
+			addOutputSelector(virtuaPartitionNodes.get(vertexID).f0, outputSelector);
+		} else if (virtualSelectNodes.containsKey(vertexID)) {
+			addOutputSelector(virtualSelectNodes.get(vertexID).f0, outputSelector);
+		} else {
+			getStreamNode(vertexID).addOutputSelector(outputSelector);
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Outputselector set for {}", vertexID);
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Outputselector set for {}", vertexID);
+			}
 		}
 
 	}
 
 	public void setParallelism(Integer vertexID, int parallelism) {
-		getStreamNode(vertexID).setParallelism(parallelism);
+		if (getStreamNode(vertexID) != null) {
+			getStreamNode(vertexID).setParallelism(parallelism);
+		}
 	}
 
 	public void setKey(Integer vertexID, KeySelector<?, ?> key) {
@@ -437,17 +368,19 @@ public class StreamGraph extends StreamingPlan {
 	}
 
 	public void setBufferTimeout(Integer vertexID, long bufferTimeout) {
-		getStreamNode(vertexID).setBufferTimeout(bufferTimeout);
+		if (getStreamNode(vertexID) != null) {
+			getStreamNode(vertexID).setBufferTimeout(bufferTimeout);
+		}
 	}
 
-	private void setSerializers(Integer vertexID, TypeSerializer<?> in1, TypeSerializer<?> in2, TypeSerializer<?> out) {
+	public void setSerializers(Integer vertexID, TypeSerializer<?> in1, TypeSerializer<?> in2, TypeSerializer<?> out) {
 		StreamNode vertex = getStreamNode(vertexID);
 		vertex.setSerializerIn1(in1);
 		vertex.setSerializerIn2(in2);
 		vertex.setSerializerOut(out);
 	}
 
-	private void setSerializersFrom(Integer from, Integer to) {
+	public void setSerializersFrom(Integer from, Integer to) {
 		StreamNode fromVertex = getStreamNode(from);
 		StreamNode toVertex = getStreamNode(to);
 
@@ -469,6 +402,10 @@ public class StreamGraph extends StreamingPlan {
 
 	public void setResourceStrategy(Integer vertexID, ResourceStrategy strategy) {
 		StreamNode node = getStreamNode(vertexID);
+		if (node == null) {
+			return;
+		}
+
 		switch (strategy) {
 		case ISOLATE:
 			node.isolateSlot();
@@ -506,6 +443,11 @@ public class StreamGraph extends StreamingPlan {
 		return sources;
 	}
 
+
+	public Collection<Integer> getSinkIDs() {
+		return sinks;
+	}
+
 	public Collection<StreamNode> getStreamNodes() {
 		return streamNodes.values();
 	}
@@ -519,20 +461,44 @@ public class StreamGraph extends StreamingPlan {
 		return operatorSet;
 	}
 
-	public Collection<StreamLoop> getStreamLoops() {
-		return streamLoops.values();
+	public String getBrokerID(Integer vertexID) {
+		return vertexIDtoBrokerID.get(vertexID);
 	}
 
-	public Integer getLoopID(Integer vertexID) {
-		return vertexIDtoLoop.get(vertexID).getID();
+	public long getLoopTimeout(Integer vertexID) {
+		return vertexIDtoLoopTimeout.get(vertexID);
 	}
 
-	public String getBrokerID(Integer vertexID) {
-		return vertexIDtoBrokerID.get(vertexID);
+	public  Tuple2<StreamNode, StreamNode> createIterationSourceAndSink(int loopId, int sourceId, int sinkId, long timeout, int parallelism) {
+
+		StreamNode source = this.addNode(sourceId,
+				StreamIterationHead.class,
+				null,
+				null);
+		sources.add(source.getId());
+		setParallelism(source.getId(), parallelism);
+
+		StreamNode sink = this.addNode(sinkId,
+				StreamIterationTail.class,
+				null,
+				null);
+		sinks.add(sink.getId());
+		setParallelism(sink.getId(), parallelism);
+
+		iterationSourceSinkPairs.add(new Tuple2<StreamNode, StreamNode>(source, sink));
+
+		source.setOperatorName("IterationSource-" + loopId);
+		sink.setOperatorName("IterationSink-" + loopId);
+		this.vertexIDtoBrokerID.put(source.getId(), "broker-" + loopId);
+		this.vertexIDtoBrokerID.put(sink.getId(), "broker-" + loopId);
+		this.vertexIDtoLoopTimeout.put(source.getId(), timeout);
+		this.vertexIDtoLoopTimeout.put(sink.getId(), timeout);
+
+		return new Tuple2<StreamNode, StreamNode>(source, sink);
 	}
 
-	public long getLoopTimeout(Integer vertexID) {
-		return vertexIDtoLoop.get(vertexID).getTimeout();
+	public Set<Tuple2<StreamNode, StreamNode>> getIterationSourceSinkPairs() {
+		return iterationSourceSinkPairs;
 	}
 
 	protected void removeEdge(StreamEdge edge) {
@@ -570,7 +536,6 @@ public class StreamGraph extends StreamingPlan {
 	 *            name of the jobGraph
 	 */
 	public JobGraph getJobGraph(String jobGraphName) {
-		finalizeLoops();
 		// temporarily forbid checkpointing for iterative jobs
 		if (isIterative() && isCheckpointingEnabled() && !forceCheckpoint) {
 			throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
new file mode 100644
index 0000000..6df8cb5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -0,0 +1,530 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.graph;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.FileSourceFunction;
+import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.transformations.SelectTransformation;
+import org.apache.flink.streaming.api.transformations.SinkTransformation;
+import org.apache.flink.streaming.api.transformations.SourceTransformation;
+import org.apache.flink.streaming.api.transformations.SplitTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.streaming.api.transformations.UnionTransformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A generator that generates a {@link StreamGraph} from a graph of
+ * {@link StreamTransformation StreamTransformations}.
+ *
+ * <p>
+ * This traverses the tree of {@code StreamTransformations} starting from the sinks. At each
+ * we transformation recursively transform the inputs, then create a node in the {@code StreamGraph}
+ * and add edges from the input Nodes to our newly created node. The transformation methods
+ * return the IDs of the nodes in the StreamGraph that represent the input transformation. Several
+ * IDs can be returned to be able to deal with feedback transformations and unions.
+ *
+ * <p>
+ * Partitioning, split/select and union don't create actual nodes in the {@code StreamGraph}. For
+ * these, we create a virtual node in the {@code StreamGraph} that holds the specific property, i.e.
+ * partitioning, selector and so on. When an edge is created from a virtual node to a downstream
+ * node the {@code StreamGraph} resolved the id of the original node and creates an edge
+ * in the graph with the desired property. For example, if you have this graph:
+ *
+ * <pre>
+ *     Map-1 -> HashPartition-2 -> Map-3
+ * </pre>
+ *
+ * where the numbers represent transformation IDs. We first recurse all the way down. {@code Map-1}
+ * is transformed, i.e. we create a {@code StreamNode} with ID 1. Then we transform the
+ * {@code HashPartition}, for this, we create virtual node of ID 4 that holds the property
+ * {@code HashPartition}. This transformation returns the ID 4. Then we transform the {@code Map-3}.
+ * We add the edge {@code 4 -> 3}. The {@code StreamGraph} resolved the actual node with ID 1 and
+ * creates and edge {@code 1 -> 3} with the property HashPartition.
+ */
+public class StreamGraphGenerator {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StreamGraphGenerator.class);
+
+	// The StreamGraph that is being built, this is initialized at the beginning.
+	private StreamGraph streamGraph;
+
+	private final StreamExecutionEnvironment env;
+
+	// This is used to assign a unique ID to iteration source/sink
+	protected static Integer iterationIdCounter = 0;
+	public static int getNewIterationNodeId() {
+		iterationIdCounter--;
+		return iterationIdCounter;
+	}
+
+	// Keep track of which Transforms we have already transformed, this is necessary because
+	// we have loops, i.e. feedback edges.
+	private Map<StreamTransformation<?>, Collection<Integer>> alreadyTransformed;
+
+
+	/**
+	 * Private constructor. The generator should only be invoked using {@link #generate}.
+	 */
+	private StreamGraphGenerator(StreamExecutionEnvironment env) {
+		this.streamGraph = new StreamGraph(env);
+		this.streamGraph.setChaining(env.isChainingEnabled());
+		if (env.getCheckpointInterval() > 0) {
+			this.streamGraph.setCheckpointingEnabled(true);
+			this.streamGraph.setCheckpointingInterval(env.getCheckpointInterval());
+			this.streamGraph.setCheckpointingMode(env.getCheckpointingMode());
+		}
+		this.streamGraph.setStateHandleProvider(env.getStateHandleProvider());
+		if (env.isForceCheckpointing()) {
+			this.streamGraph.forceCheckpoint();
+		}
+		this.env = env;
+		this.alreadyTransformed = Maps.newHashMap();
+	}
+
+	/**
+	 * Generates a {@code StreamGraph} by traversing the graph of {@code StreamTransformations}
+	 * starting from the given transformations.
+	 *
+	 * @param env The {@code StreamExecutionEnvironment} that is used to set some parameters of the
+	 *            job
+	 * @param transformations The transformations starting from which to transform the graph
+	 *
+	 * @return The generated {@code StreamGraph}
+	 */
+	public static StreamGraph generate(StreamExecutionEnvironment env, List<StreamTransformation<?>> transformations) {
+		return new StreamGraphGenerator(env).generateInternal(transformations);
+	}
+
+	/**
+	 * This starts the actual transformation, beginning from the sinks.
+	 */
+	private StreamGraph generateInternal(List<StreamTransformation<?>> transformations) {
+		for (StreamTransformation<?> transformation: transformations) {
+			transform(transformation);
+		}
+		return streamGraph;
+	}
+
+	/**
+	 * Transforms one {@code StreamTransformation}.
+	 *
+	 * <p>
+	 * This checks whether we already transformed it and exits early in that case. If not it
+	 * delegates to one of the transformation specific methods.
+	 */
+	private Collection<Integer> transform(StreamTransformation<?> transform) {
+
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
+		LOG.debug("Transforming " + transform);
+
+		// call at least once to trigger exceptions about MissingTypeInfo
+		transform.getOutputType();
+
+		Collection<Integer> transformedIds;
+		if (transform instanceof OneInputTransformation<?, ?>) {
+			transformedIds = transformOnInputTransform((OneInputTransformation<?, ?>) transform);
+		} else if (transform instanceof TwoInputTransformation<?, ?, ?>) {
+			transformedIds = transformTwoInputTransform((TwoInputTransformation<?, ?, ?>) transform);
+		} else if (transform instanceof SourceTransformation<?>) {
+			transformedIds = transformSource((SourceTransformation<?>) transform);
+		} else if (transform instanceof SinkTransformation<?>) {
+			transformedIds = transformSink((SinkTransformation<?>) transform);
+		} else if (transform instanceof UnionTransformation<?>) {
+			transformedIds = transformUnion((UnionTransformation<?>) transform);
+		} else if (transform instanceof SplitTransformation<?>) {
+			transformedIds = transformSplit((SplitTransformation<?>) transform);
+		} else if (transform instanceof SelectTransformation<?>) {
+			transformedIds = transformSelect((SelectTransformation<?>) transform);
+		} else if (transform instanceof FeedbackTransformation<?>) {
+			transformedIds = transformFeedback((FeedbackTransformation<?>) transform);
+		} else if (transform instanceof CoFeedbackTransformation<?>) {
+			transformedIds = transformCoFeedback((CoFeedbackTransformation<?>) transform);
+		} else if (transform instanceof PartitionTransformation<?>) {
+			transformedIds = transformPartition((PartitionTransformation<?>) transform);
+		} else {
+			throw new IllegalStateException("Unknown transformation: " + transform);
+		}
+
+		// need this check because the iterate transformation adds itself before
+		// transforming the feedback edges
+		if (!alreadyTransformed.containsKey(transform)) {
+			alreadyTransformed.put(transform, transformedIds);
+		}
+
+		if (transform.getBufferTimeout() > 0) {
+			streamGraph.setBufferTimeout(transform.getId(), transform.getBufferTimeout());
+		}
+		if (transform.getResourceStrategy() != StreamGraph.ResourceStrategy.DEFAULT) {
+			streamGraph.setResourceStrategy(transform.getId(), transform.getResourceStrategy());
+		}
+
+		return transformedIds;
+	}
+
+	/**
+	 * Transforms a {@code UnionTransformation}.
+	 *
+	 * <p>
+	 * This is easy, we only have to transform the inputs and return all the IDs in a list so
+	 * that downstream operations can connect to all upstream nodes.
+	 */
+	private <T> Collection<Integer> transformUnion(UnionTransformation<T> union) {
+		List<StreamTransformation<T>> inputs = union.getInputs();
+		List<Integer> resultIds = Lists.newArrayList();
+
+		for (StreamTransformation<T> input: inputs) {
+			resultIds.addAll(transform(input));
+		}
+
+		return resultIds;
+	}
+
+	/**
+	 * Transforms a {@code PartitionTransformation}.
+	 *
+	 * <p>
+	 * For this we create a virtual node in the {@code StreamGraph} that holds the partition
+	 * property. @see StreamGraphGenerator
+	 */
+	private <T> Collection<Integer> transformPartition(PartitionTransformation<T> partition) {
+		StreamTransformation<T> input = partition.getInput();
+		List<Integer> resultIds = Lists.newArrayList();
+
+		Collection<Integer> transformedIds = transform(input);
+		for (Integer transformedId: transformedIds) {
+			int virtualId = StreamTransformation.getNewNodeId();
+			streamGraph.addVirtualPartitionNode(transformedId, virtualId, partition.getPartitioner());
+			resultIds.add(virtualId);
+		}
+
+		return resultIds;
+	}
+
+	/**
+	 * Transforms a {@code SplitTransformation}.
+	 *
+	 * <p>
+	 * We add the output selector to previously transformed nodes.
+	 */
+	private <T> Collection<Integer> transformSplit(SplitTransformation<T> split) {
+
+		StreamTransformation<T> input = split.getInput();
+		Collection<Integer> resultIds = transform(input);
+
+		// the recursive transform call might have transformed this already
+		if (alreadyTransformed.containsKey(split)) {
+			return alreadyTransformed.get(split);
+		}
+
+		for (int inputId : resultIds) {
+			streamGraph.addOutputSelector(inputId, split.getOutputSelector());
+		}
+
+
+		return resultIds;
+	}
+
+	/**
+	 * Transforms a {@code SelectTransformation}.
+	 *
+	 * <p>
+	 * For this we create a virtual node in the {@code StreamGraph} holds the selected names.
+	 * @see org.apache.flink.streaming.api.graph.StreamGraphGenerator
+	 */
+	private <T> Collection<Integer> transformSelect(SelectTransformation<T> select) {
+		StreamTransformation<T> input = select.getInput();
+		Collection<Integer> resultIds = transform(input);
+
+
+		// the recursive transform might have already transformed this
+		if (alreadyTransformed.containsKey(select)) {
+			return alreadyTransformed.get(select);
+		}
+
+		List<Integer> virtualResultIds = Lists.newArrayList();
+
+		for (int inputId : resultIds) {
+			int virtualId = StreamTransformation.getNewNodeId();
+			streamGraph.addVirtualSelectNode(inputId, virtualId, select.getSelectedNames());
+			virtualResultIds.add(virtualId);
+		}
+		return virtualResultIds;
+	}
+
+	/**
+	 * Transforms a {@code FeedbackTransformation}.
+	 *
+	 * <p>
+	 * This will recursively transform the input and the feedback edges. We return the concatenation
+	 * of the input IDs and the feedback IDs so that downstream operations can be wired to both.
+	 *
+	 * <p>
+	 * This is responsible for creating the IterationSource and IterationSink which
+	 * are used to feed back the elements.
+	 */
+	private <T> Collection<Integer> transformFeedback(FeedbackTransformation<T> iterate) {
+
+		if (iterate.getFeedbackEdges().size() <= 0) {
+			throw new IllegalStateException("Iteration " + iterate + " does not have any feedback edges.");
+		}
+
+		StreamTransformation<T> input = iterate.getInput();
+		List<Integer> resultIds = Lists.newArrayList();
+
+		// first transform the input stream(s) and store the result IDs
+		resultIds.addAll(transform(input));
+
+		// the recursive transform might have already transformed this
+		if (alreadyTransformed.containsKey(iterate)) {
+			return alreadyTransformed.get(iterate);
+		}
+
+
+		// create the fake iteration source/sink pair
+		Tuple2<StreamNode, StreamNode> itSourceAndSink = streamGraph.createIterationSourceAndSink(
+				iterate.getId(),
+				getNewIterationNodeId(),
+				getNewIterationNodeId(),
+				iterate.getWaitTime(),
+				iterate.getParallelism());
+
+		StreamNode itSource = itSourceAndSink.f0;
+		StreamNode itSink = itSourceAndSink.f1;
+
+		// We set the proper serializers for the sink/source
+		streamGraph.setSerializers(itSource.getId(), null, null, iterate.getOutputType().createSerializer(env.getConfig()));
+		streamGraph.setSerializers(itSink.getId(), iterate.getOutputType().createSerializer(env.getConfig()), null, null);
+
+		// also add the feedback source ID to the result IDs, so that downstream operators will
+		// add both as input
+		resultIds.add(itSource.getId());
+
+		// at the iterate to the already-seen-set with the result IDs, so that we can transform
+		// the feedback edges and let them stop when encountering the iterate node
+		alreadyTransformed.put(iterate, resultIds);
+
+		for (StreamTransformation<T> feedbackEdge : iterate.getFeedbackEdges()) {
+			Collection<Integer> feedbackIds = transform(feedbackEdge);
+			for (Integer feedbackId: feedbackIds) {
+				streamGraph.addEdge(feedbackId,
+						itSink.getId(),
+						0
+				);
+			}
+		}
+
+		return resultIds;
+	}
+
+	/**
+	 * Transforms a {@code CoFeedbackTransformation}.
+	 *
+	 * <p>
+	 * This will only transform feedback edges, the result of this transform will be wired
+	 * to the second input of a Co-Transform. The original input is wired directly to the first
+	 * input of the downstream Co-Transform.
+	 *
+	 * <p>
+	 * This is responsible for creating the IterationSource and IterationSink which
+	 * are used to feed back the elements.
+	 */
+	private <F> Collection<Integer> transformCoFeedback(CoFeedbackTransformation<F> coIterate) {
+
+		// For Co-Iteration we don't need to transform the input and wire the input to the
+		// head operator by returning the input IDs, the input is directly wired to the left
+		// input of the co-operation. This transform only needs to return the ids of the feedback
+		// edges, since they need to be wired to the second input of the co-operation.
+
+		// create the fake iteration source/sink pair
+		Tuple2<StreamNode, StreamNode> itSourceAndSink = streamGraph.createIterationSourceAndSink(
+				coIterate.getId(),
+				getNewIterationNodeId(),
+				getNewIterationNodeId(),
+				coIterate.getWaitTime(),
+				coIterate.getParallelism());
+
+		StreamNode itSource = itSourceAndSink.f0;
+		StreamNode itSink = itSourceAndSink.f1;
+
+		// We set the proper serializers for the sink/source
+		streamGraph.setSerializers(itSource.getId(), null, null, coIterate.getOutputType().createSerializer(env.getConfig()));
+		streamGraph.setSerializers(itSink.getId(), coIterate.getOutputType().createSerializer(env.getConfig()), null, null);
+
+		Collection<Integer> resultIds = Collections.singleton(itSource.getId());
+
+		// at the iterate to the already-seen-set with the result IDs, so that we can transform
+		// the feedback edges and let them stop when encountering the iterate node
+		alreadyTransformed.put(coIterate, resultIds);
+
+		for (StreamTransformation<F> feedbackEdge : coIterate.getFeedbackEdges()) {
+			Collection<Integer> feedbackIds = transform(feedbackEdge);
+			for (Integer feedbackId: feedbackIds) {
+				streamGraph.addEdge(feedbackId,
+						itSink.getId(),
+						0
+				);
+			}
+		}
+
+		return Collections.singleton(itSource.getId());
+	}
+
+	/**
+	 * Transforms a {@code SourceTransformation}.
+	 */
+	private <T> Collection<Integer> transformSource(SourceTransformation<T> source) {
+		streamGraph.addSource(source.getId(),
+				source.getOperator(),
+				null,
+				source.getOutputType(),
+				"Source: " + source.getName());
+		if (source.getOperator().getUserFunction() instanceof FileSourceFunction) {
+			FileSourceFunction<T> fs = (FileSourceFunction<T>) source.getOperator().getUserFunction();
+			streamGraph.setInputFormat(source.getId(), fs.getFormat());
+		}
+		streamGraph.setParallelism(source.getId(), source.getParallelism());
+		return Collections.singleton(source.getId());
+	}
+
+	/**
+	 * Transforms a {@code SourceTransformation}.
+	 */
+	private <T> Collection<Integer> transformSink(SinkTransformation<T> sink) {
+
+		Collection<Integer> inputIds = transform(sink.getInput());
+
+		streamGraph.addSink(sink.getId(),
+				sink.getOperator(),
+				sink.getInput().getOutputType(),
+				null,
+				"Sink: " + sink.getName());
+
+		streamGraph.setParallelism(sink.getId(), sink.getParallelism());
+
+		for (Integer inputId: inputIds) {
+			streamGraph.addEdge(inputId,
+					sink.getId(),
+					0
+			);
+		}
+
+
+		if (sink.getStateKeySelector() != null) {
+			streamGraph.setKey(sink.getId(), sink.getStateKeySelector());
+		}
+
+		return Collections.emptyList();
+	}
+
+	/**
+	 * Transforms a {@code OneInputTransformation}.
+	 *
+	 * <p>
+	 * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and
+	 * wired the inputs to this new node.
+	 */
+	private <IN, OUT> Collection<Integer> transformOnInputTransform(OneInputTransformation<IN, OUT> transform) {
+
+		Collection<Integer> inputIds = transform(transform.getInput());
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
+		streamGraph.addOperator(transform.getId(),
+				transform.getOperator(),
+				transform.getInputType(),
+				transform.getOutputType(),
+				transform.getName());
+
+		if (transform.getStateKeySelector() != null) {
+			streamGraph.setKey(transform.getId(), transform.getStateKeySelector());
+		}
+
+		streamGraph.setParallelism(transform.getId(), transform.getParallelism());
+
+		for (Integer inputId: inputIds) {
+			streamGraph.addEdge(inputId, transform.getId(), 0);
+		}
+
+		return Collections.singleton(transform.getId());
+	}
+
+	/**
+	 * Transforms a {@code TwoInputTransformation}.
+	 *
+	 * <p>
+	 * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and
+	 * wired the inputs to this new node.
+	 */
+	private <IN1, IN2, OUT> Collection<Integer> transformTwoInputTransform(TwoInputTransformation<IN1, IN2, OUT> transform) {
+
+		Collection<Integer> inputIds1 = transform(transform.getInput1());
+		Collection<Integer> inputIds2 = transform(transform.getInput2());
+
+		// the recursive call might have already transformed this
+		if (alreadyTransformed.containsKey(transform)) {
+			return alreadyTransformed.get(transform);
+		}
+
+		streamGraph.addCoOperator(
+				transform.getId(),
+				transform.getOperator(),
+				transform.getInputType1(),
+				transform.getInputType2(),
+				transform.getOutputType(),
+				transform.getName());
+
+		streamGraph.setParallelism(transform.getId(), transform.getParallelism());
+
+		for (Integer inputId: inputIds1) {
+			streamGraph.addEdge(inputId,
+					transform.getId(),
+					1
+			);
+		}
+
+		for (Integer inputId: inputIds2) {
+			streamGraph.addEdge(inputId,
+					transform.getId(),
+					2
+			);
+		}
+
+		return Collections.singleton(transform.getId());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java
deleted file mode 100644
index ba987ef..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamLoop.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-
-/**
- * Object for representing loops in streaming programs.
- * 
- */
-public class StreamLoop {
-
-	private Integer loopID;
-
-	private List<StreamNode> headOperators = new ArrayList<StreamNode>();
-	private List<StreamNode> tailOperators = new ArrayList<StreamNode>();
-	private List<StreamPartitioner<?>> tailPartitioners = new ArrayList<StreamPartitioner<?>>();
-	private List<List<String>> tailSelectedNames = new ArrayList<List<String>>();
-
-	private boolean coIteration = false;
-	private TypeInformation<?> feedbackType = null;
-
-	private long timeout;
-	private boolean tailPartitioning = false;
-
-	private List<Tuple2<StreamNode, StreamNode>> sourcesAndSinks = new ArrayList<Tuple2<StreamNode, StreamNode>>();
-
-	public StreamLoop(Integer loopID, long timeout, TypeInformation<?> feedbackType) {
-		this.loopID = loopID;
-		this.timeout = timeout;
-		if (feedbackType != null) {
-			this.feedbackType = feedbackType;
-			coIteration = true;
-			tailPartitioning = true;
-		}
-	}
-
-	public Integer getID() {
-		return loopID;
-	}
-
-	public long getTimeout() {
-		return timeout;
-	}
-
-	public boolean isCoIteration() {
-		return coIteration;
-	}
-
-	public TypeInformation<?> getFeedbackType() {
-		return feedbackType;
-	}
-
-	public void addSourceSinkPair(StreamNode source, StreamNode sink) {
-		this.sourcesAndSinks.add(new Tuple2<StreamNode, StreamNode>(source, sink));
-	}
-
-	public List<Tuple2<StreamNode, StreamNode>> getSourceSinkPairs() {
-		return this.sourcesAndSinks;
-	}
-
-	public void addHeadOperator(StreamNode head) {
-		this.headOperators.add(head);
-	}
-
-	public void addTailOperator(StreamNode tail, StreamPartitioner<?> partitioner,
-			List<String> selectedNames) {
-		this.tailOperators.add(tail);
-		this.tailPartitioners.add(partitioner);
-		this.tailSelectedNames.add(selectedNames);
-	}
-
-	public void applyTailPartitioning() {
-		this.tailPartitioning = true;
-	}
-
-	public boolean keepsPartitioning() {
-		return tailPartitioning;
-	}
-
-	public List<StreamNode> getHeads() {
-		return headOperators;
-	}
-
-	public List<StreamNode> getTails() {
-		return tailOperators;
-	}
-
-	public List<StreamPartitioner<?>> getTailPartitioners() {
-		return tailPartitioners;
-	}
-
-	public List<List<String>> getTailSelectedNames() {
-		return tailSelectedNames;
-	}
-
-	@Override
-	public String toString() {
-		return "ID: " + loopID + "\n" + "Head: " + headOperators + "\n" + "Tail: " + tailOperators
-				+ "\n" + "TP: " + tailPartitioners + "\n" + "TSN: " + tailSelectedNames;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
index 62e2d83..9110cd3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
@@ -125,7 +125,11 @@ public class StreamNode implements Serializable {
 	}
 
 	public int getParallelism() {
-		return parallelism != null ? parallelism : env.getParallelism();
+		if (parallelism == -1) {
+			return env.getParallelism();
+		} else {
+			return parallelism;
+		}
 	}
 
 	public void setParallelism(Integer parallelism) {
@@ -218,7 +222,7 @@ public class StreamNode implements Serializable {
 	
 	@Override
 	public String toString() {
-		return operatorName + id;
+		return operatorName + "-" + id;
 	}
 
 	public KeySelector<?, ?> getStatePartitioner() {
@@ -228,4 +232,23 @@ public class StreamNode implements Serializable {
 	public void setStatePartitioner(KeySelector<?, ?> statePartitioner) {
 		this.statePartitioner = statePartitioner;
 	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		StreamNode that = (StreamNode) o;
+
+		return id.equals(that.id);
+	}
+
+	@Override
+	public int hashCode() {
+		return id.hashCode();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index 5280fb2..314d1b3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -44,8 +44,8 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner.PartitioningStrategy;
 import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
 import org.apache.flink.streaming.runtime.tasks.StreamIterationTail;
 import org.apache.flink.util.InstantiationUtil;
@@ -294,11 +294,6 @@ public class StreamingJobGraphGenerator {
 		List<StreamEdge> allOutputs = new ArrayList<StreamEdge>(chainableOutputs);
 		allOutputs.addAll(nonChainableOutputs);
 
-		for (StreamEdge output : allOutputs) {
-			config.setSelectedNames(output.getTargetId(),
-					streamGraph.getStreamEdge(vertexID, output.getTargetId()).getSelectedNames());
-		}
-
 		vertexConfigs.put(vertexID, config);
 	}
 
@@ -316,7 +311,7 @@ public class StreamingJobGraphGenerator {
 		downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1);
 
 		StreamPartitioner<?> partitioner = edge.getPartitioner();
-		if (partitioner.getStrategy() == PartitioningStrategy.FORWARD) {
+		if (partitioner instanceof ForwardPartitioner) {
 			downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE);
 		} else {
 			downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.ALL_TO_ALL);
@@ -345,7 +340,7 @@ public class StreamingJobGraphGenerator {
 				&& (headOperator.getChainingStrategy() == ChainingStrategy.HEAD ||
 					headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS ||
 					headOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS)
-				&& (edge.getPartitioner().getStrategy() == PartitioningStrategy.FORWARD || downStreamVertex
+				&& (edge.getPartitioner() instanceof ForwardPartitioner || downStreamVertex
 						.getParallelism() == 1)
 				&& upStreamVertex.getParallelism() == downStreamVertex.getParallelism()
 				&& (streamGraph.isChainingEnabled() ||
@@ -370,21 +365,19 @@ public class StreamingJobGraphGenerator {
 			}
 		}
 
-		for (StreamLoop loop : streamGraph.getStreamLoops()) {
-			for (Tuple2<StreamNode, StreamNode> pair : loop.getSourceSinkPairs()) {
-				
-				CoLocationGroup ccg = new CoLocationGroup();
-				
-				JobVertex source = jobVertices.get(pair.f0.getId());
-				JobVertex sink = jobVertices.get(pair.f1.getId());
-				
-				ccg.addVertex(source);
-				ccg.addVertex(sink);
-				source.updateCoLocationGroup(ccg);
-				sink.updateCoLocationGroup(ccg);
-			}
+		for (Tuple2<StreamNode, StreamNode> pair : streamGraph.getIterationSourceSinkPairs()) {
 
+			CoLocationGroup ccg = new CoLocationGroup();
+
+			JobVertex source = jobVertices.get(pair.f0.getId());
+			JobVertex sink = jobVertices.get(pair.f1.getId());
+
+			ccg.addVertex(source);
+			ccg.addVertex(sink);
+			source.updateCoLocationGroup(ccg);
+			sink.updateCoLocationGroup(ccg);
 		}
+
 	}
 	
 	private void configureCheckpointing() {

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
index dce7684..cbd2a40 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
@@ -28,7 +28,6 @@ import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer;
 import org.apache.flink.streaming.api.operators.windowing.WindowFlattener;
 import org.apache.flink.streaming.api.operators.windowing.WindowMerger;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
 
 public class WindowingOptimizer {
 
@@ -64,8 +63,7 @@ public class WindowingOptimizer {
 				StreamNode mergeInput = input.getInEdges().get(0).getSourceVertex();
 
 				// We connect the merge input to the flattener directly
-				streamGraph.addEdge(mergeInput.getId(), flattenerId,
-						new RebalancePartitioner(true), 0, new ArrayList<String>());
+				streamGraph.addEdge(mergeInput.getId(), flattenerId, 0);
 
 				// If the merger is only connected to the flattener we delete it
 				// completely, otherwise we only remove the edge
@@ -107,8 +105,7 @@ public class WindowingOptimizer {
 
 				for (StreamEdge edge1 : discretizer.getInEdges()) {
 					for (StreamEdge edge2 : candidate.f1.get(0).getInEdges()) {
-						if (edge1.getPartitioner().getStrategy() != edge2.getPartitioner()
-								.getStrategy()) {
+						if (edge1.getPartitioner().getClass() != edge2.getPartitioner().getClass()) {
 							partitionersMatch = false;
 						}
 					}
@@ -155,8 +152,7 @@ public class WindowingOptimizer {
 		for (int i = 0; i < numOutputs; i++) {
 			StreamEdge outEdge = outEdges.get(i);
 
-			streamGraph.addEdge(replaceWithId, outEdge.getTargetId(), outEdge.getPartitioner(), 0,
-					new ArrayList<String>());
+			streamGraph.addEdge(replaceWithId, outEdge.getTargetId(), 0);
 		}
 
 		// Remove the other discretizer

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
new file mode 100644
index 0000000..67ccbd6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This represents a feedback point in a topology. The type of the feedback elements must not match
+ * the type of the upstream {@code StreamTransformation} because the only allowed operations
+ * after a {@code CoFeedbackTransformation} are
+ * {@link org.apache.flink.streaming.api.transformations.TwoInputTransformation TwoInputTransformations}.
+ * The upstream {@code StreamTransformation} will be connected to the first input of the Co-Transform
+ * while the feedback edges will be connected to the second input.
+ *
+ * <p>
+ * Both the partitioning of the input and the feedback edges is preserved. They can also have
+ * differing partitioning strategies. This requires, however, that the parallelism of the feedback
+ * {@code StreamTransformations} must match the parallelism of the input
+ * {@code StreamTransformation}.
+ *
+ * <p>
+ * The upstream {@code StreamTransformation} is not wired to this {@code CoFeedbackTransformation}.
+ * It is instead directly wired to the {@code TwoInputTransformation} after this
+ * {@code CoFeedbackTransformation}.
+ *
+ * <p>
+ * This is different from Iterations in batch processing.
+ * @see org.apache.flink.streaming.api.transformations.FeedbackTransformation
+ *
+ * @param <F> The type of the feedback elements.
+ *
+ */
+public class CoFeedbackTransformation<F> extends StreamTransformation<F> {
+
+	private final List<StreamTransformation<F>> feedbackEdges;
+
+	private final Long waitTime;
+
+	/**
+	 * Creates a new {@code CoFeedbackTransformation} from the given input.
+	 *
+	 * @param parallelism The parallelism of the upstream {@code StreamTransformatino} and the
+	 *                    feedback edges.
+	 * @param feedbackType The type of the feedback edges
+	 * @param waitTime The wait time of the feedback operator. After the time expires
+	 *                          the operation will close and not receive any more feedback elements.
+	 */
+	public CoFeedbackTransformation(int parallelism,
+			TypeInformation<F> feedbackType,
+			Long waitTime) {
+		super("CoFeedback", feedbackType, parallelism);
+		this.waitTime = waitTime;
+		this.feedbackEdges = Lists.newArrayList();
+	}
+
+	/**
+	 * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match
+	 * the parallelism of the input {@code StreamTransformation} of the upstream
+	 * {@code StreamTransformation}.
+	 *
+	 * @param transform The new feedback {@code StreamTransformation}.
+	 */
+	public void addFeedbackEdge(StreamTransformation<F> transform) {
+
+		if (transform.getParallelism() != this.getParallelism()) {
+			throw new UnsupportedOperationException(
+					"Parallelism of the feedback stream must match the parallelism of the original" +
+							" stream. Parallelism of original stream: " + this.getParallelism() +
+							"; parallelism of feedback stream: " + transform.getParallelism());
+		}
+
+		feedbackEdges.add(transform);
+	}
+
+	/**
+	 * Returns the list of feedback {@code StreamTransformations}.
+	 */
+	public List<StreamTransformation<F>> getFeedbackEdges() {
+		return feedbackEdges;
+	}
+
+	/**
+	 * Returns the wait time. This is the amount of time that the feedback operator keeps listening
+	 * for feedback elements. Once the time expires the operation will close and will not receive
+	 * further elements.
+	 */
+	public Long getWaitTime() {
+		return waitTime;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		return Collections.<StreamTransformation<?>>singleton(this);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
new file mode 100644
index 0000000..11a2f33
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This represents a feedback point in a topology.
+ *
+ * <p>
+ * This is different from how iterations work in batch processing. Once a feedback point is defined
+ * you can connect one or several {@code StreamTransformations} as a feedback edges. Operations
+ * downstream from the feedback point will receive elements from the input of this feedback point
+ * and from the feedback edges.
+ *
+ * <p>
+ * Both the partitioning of the input and the feedback edges is preserved. They can also have
+ * differing partitioning strategies. This requires, however, that the parallelism of the feedback
+ * {@code StreamTransformations} must match the parallelism of the input
+ * {@code StreamTransformation}.
+ *
+ * <p>
+ * The type of the input {@code StreamTransformation} and the feedback {@code StreamTransformation}
+ * must match.
+ *
+ * @param <T> The type of the input elements and the feedback elements.
+ */
+public class FeedbackTransformation<T> extends StreamTransformation<T> {
+
+	private final StreamTransformation<T> input;
+
+	private final List<StreamTransformation<T>> feedbackEdges;
+
+	private final Long waitTime;
+
+	/**
+	 * Creates a new {@code FeedbackTransformation} from the given input.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param waitTime The wait time of the feedback operator. After the time expires
+	 *                          the operation will close and not receive any more feedback elements.
+	 */
+	public FeedbackTransformation(StreamTransformation<T> input, Long waitTime) {
+		super("Feedback", input.getOutputType(), input.getParallelism());
+		this.input = input;
+		this.waitTime = waitTime;
+		this.feedbackEdges = Lists.newArrayList();
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation} of this {@code FeedbackTransformation}.
+	 */
+	public StreamTransformation<T> getInput() {
+		return input;
+	}
+
+	/**
+	 * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match
+	 * the parallelism of the input {@code StreamTransformation} of this
+	 * {@code FeedbackTransformation}
+	 *
+	 * @param transform The new feedback {@code StreamTransformation}.
+	 */
+	public void addFeedbackEdge(StreamTransformation<T> transform) {
+
+		if (transform.getParallelism() != this.getParallelism()) {
+			throw new UnsupportedOperationException(
+					"Parallelism of the feedback stream must match the parallelism of the original" +
+							" stream. Parallelism of original stream: " + this.getParallelism() +
+							"; parallelism of feedback stream: " + transform.getParallelism());
+		}
+
+		feedbackEdges.add(transform);
+	}
+
+	/**
+	 * Returns the list of feedback {@code StreamTransformations}.
+	 */
+	public List<StreamTransformation<T>> getFeedbackEdges() {
+		return feedbackEdges;
+	}
+
+	/**
+	 * Returns the wait time. This is the amount of time that the feedback operator keeps listening
+	 * for feedback elements. Once the time expires the operation will close and will not receive
+	 * further elements.
+	 */
+	public Long getWaitTime() {
+		return waitTime;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
new file mode 100644
index 0000000..945d8eb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This Transformation represents the application of a
+ * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} to one input
+ * {@link org.apache.flink.streaming.api.transformations.StreamTransformation}.
+ *
+ * @param <IN> The type of the elements in the nput {@code StreamTransformation}
+ * @param <OUT> The type of the elements that result from this {@code OneInputTransformation}
+ */
+public class OneInputTransformation<IN, OUT> extends StreamTransformation<OUT> {
+
+	private final StreamTransformation<IN> input;
+
+	private KeySelector<IN, ?> stateKeySelector;
+
+	private final OneInputStreamOperator<IN, OUT> operator;
+
+	/**
+	 * Creates a new {@code OneInputTransformation} from the given input and operator.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
+	 * @param operator The {@code TwoInputStreamOperator}
+	 * @param outputType The type of the elements produced by this {@code OneInputTransformation}
+	 * @param parallelism The parallelism of this {@code OneInputTransformation}
+	 */
+	public OneInputTransformation(
+			StreamTransformation<IN> input,
+			String name,
+			OneInputStreamOperator<IN, OUT> operator,
+			TypeInformation<OUT> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.input = input;
+		this.operator = operator;
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation} of this {@code OneInputTransformation}.
+	 */
+	public StreamTransformation<IN> getInput() {
+		return input;
+	}
+
+	/**
+	 * Returns the {@code TypeInformation} for the elements of the input.
+	 */
+	public TypeInformation<IN> getInputType() {
+		return input.getOutputType();
+	}
+
+	/**
+	 * Returns the {@code TwoInputStreamOperator} of this Transformation.
+	 */
+	public OneInputStreamOperator<IN, OUT> getOperator() {
+		return operator;
+	}
+
+	/**
+	 * Sets the {@link KeySelector} that must be used for partitioning keyed state of this operation.
+	 *
+	 * @param stateKeySelector The {@code KeySelector} to set
+	 */
+	public void setStateKeySelector(KeySelector<IN, ?> stateKeySelector) {
+		this.stateKeySelector = stateKeySelector;
+	}
+
+	/**
+	 * Returns the {@code KeySelector} that must be used for partitioning keyed state in this
+	 * Operation.
+	 *
+	 * @see #setStateKeySelector
+	 */
+	public KeySelector<IN, ?> getStateKeySelector() {
+		return stateKeySelector;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		operator.setChainingStrategy(strategy);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
new file mode 100644
index 0000000..1165d5d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This transformation represents a change of partitioning of the input elements.
+ *
+ * <p>
+ * This does not create a physical operation, it only affects how upstream operations are
+ * connected to downstream operations.
+ *
+ * @param <T> The type of the elements that result from this {@code PartitionTransformation}
+ */
+public class PartitionTransformation<T> extends StreamTransformation<T> {
+	private final StreamTransformation<T> input;
+	private final StreamPartitioner<T> partitioner;
+
+	/**
+	 * Creates a new {@code PartitionTransformation} from the given input and
+	 * {@link StreamPartitioner}.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param partitioner The {@code StreamPartitioner}
+	 */
+	public PartitionTransformation(StreamTransformation<T> input, StreamPartitioner<T> partitioner) {
+		super("Partition", input.getOutputType(), input.getParallelism());
+		this.input = input;
+		this.partitioner = partitioner;
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}.
+	 */
+	public StreamTransformation<T> getInput() {
+		return input;
+	}
+
+	/**
+	 * Returns the {@code StreamPartitioner} that must be used for partitioning the elements
+	 * of the input {@code StreamTransformation}.
+	 */
+	public StreamPartitioner<T> getPartitioner() {
+		return partitioner;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
new file mode 100644
index 0000000..92033bd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This transformation represents a selection of only certain upstream elements. This must
+ * follow a {@link org.apache.flink.streaming.api.transformations.SplitTransformation} that
+ * splits elements into several logical streams with assigned names.
+ *
+ * <p>
+ * This does not create a physical operation, it only affects how upstream operations are
+ * connected to downstream operations.
+ *
+ * @param <T> The type of the elements that result from this {@code SelectTransformation}
+ */
+public class SelectTransformation<T> extends StreamTransformation<T> {
+	private final StreamTransformation<T> input;
+	private List<String> selectedNames;
+
+	/**
+	 * Creates a new {@coe SelectionTransformation} from the given input that only selects
+	 * the streams with the selected names.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param selectedNames The names from the upstream {@code SplitTransformation} that this
+	 *                      {@code SelectTransformation} selects.
+	 */
+	public SelectTransformation(StreamTransformation<T> input,
+			List<String> selectedNames) {
+		super("Select", input.getOutputType(), input.getParallelism());
+		this.input = input;
+		this.selectedNames = selectedNames;
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation}.
+	 */
+	public StreamTransformation<T> getInput() {
+		return input;
+	}
+
+	/**
+	 * Returns the names of the split streams that this {@code SelectTransformation} selects.
+	 */
+	public List<String> getSelectedNames() {
+		return selectedNames;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation.");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
new file mode 100644
index 0000000..2a4e2d0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSink;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This Transformation represents a Sink.
+ *
+ * @param <T> The type of the elements in the input {@code SinkTransformation}
+ */
+public class SinkTransformation<T> extends StreamTransformation<Object> {
+
+	private final StreamTransformation<T> input;
+
+	private final StreamSink<T> operator;
+
+	// We need this because sinks can also have state that is partitioned by key
+	private KeySelector<T, ?> stateKeySelector;
+
+	/**
+	 * Creates a new {@code SinkTransformation} from the given input {@code StreamTransformation}.
+	 *
+	 * @param input The input {@code StreamTransformation}
+	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
+	 * @param operator The sink operator
+	 * @param parallelism The parallelism of this {@code SinkTransformation}
+	 */
+	public SinkTransformation(
+			StreamTransformation<T> input,
+			String name,
+			StreamSink<T> operator,
+			int parallelism) {
+		super(name, TypeExtractor.getForClass(Object.class), parallelism);
+		this.input = input;
+		this.operator = operator;
+	}
+
+	/**
+	 * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}.
+	 */
+	public StreamTransformation<T> getInput() {
+		return input;
+	}
+
+	/**
+	 * Returns the {@link StreamSink} that is the operator of this {@code SinkTransformation}.
+	 */
+	public StreamSink<T> getOperator() {
+		return operator;
+	}
+
+	/**
+	 * Sets the {@link KeySelector} that must be used for partitioning keyed state of this Sink.
+	 *
+	 * @param stateKeySelector The {@code KeySelector} to set
+	 */
+	public void setStateKeySelector(KeySelector<T, ?> stateKeySelector) {
+		this.stateKeySelector = stateKeySelector;
+	}
+
+	/**
+	 * Returns the {@code KeySelector} that must be used for partitioning keyed state in this
+	 * Sink.
+	 *
+	 * @see #setStateKeySelector
+	 */
+	public KeySelector<T, ?> getStateKeySelector() {
+		return stateKeySelector;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		List<StreamTransformation<?>> result = Lists.newArrayList();
+		result.add(this);
+		result.addAll(input.getTransitivePredecessors());
+		return result;
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		operator.setChainingStrategy(strategy);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
new file mode 100644
index 0000000..c14c58c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.transformations;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSource;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * This represents a Source. This does not actually transform anything since it has no inputs but
+ * it is the root {@code StreamTransformation} of any topology.
+ *
+ * @param <T> The type of the elements that this source produces
+ */
+public class SourceTransformation<T> extends StreamTransformation<T> {
+
+	private final StreamSource<T> operator;
+
+	/**
+	 * Creates a new {@code SourceTransformation} from the given operator.
+	 *
+	 * @param name The name of the {@code SourceTransformation}, this will be shown in Visualizations and the Log
+	 * @param operator The {@code StreamSource} that is the operator of this Transformation
+	 * @param outputType The type of the elements produced by this {@code SourceTransformation}
+	 * @param parallelism The parallelism of this {@code SourceTransformation}
+	 */
+	public SourceTransformation(
+			String name,
+			StreamSource<T> operator,
+			TypeInformation<T> outputType,
+			int parallelism) {
+		super(name, outputType, parallelism);
+		this.operator = operator;
+	}
+
+	/**
+	 * Returns the {@code StreamSource}, the operator of this {@code SourceTransformation}.
+	 */
+	public StreamSource<T> getOperator() {
+		return operator;
+	}
+
+	@Override
+	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
+		return Collections.<StreamTransformation<?>>singleton(this);
+	}
+
+	@Override
+	public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+		operator.setChainingStrategy(strategy);
+	}
+}


[6/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Posted by al...@apache.org.
[FLINK-2398][api-breaking] Introduce StreamGraphGenerator

This decouples the building of the StreamGraph from the API methods.
Before the methods would build the StreamGraph as they go. Now the API
methods build a hierachy of StreamTransformation nodes. From these a
StreamGraph is generated upon execution.

This also introduces some API breaking changes:

 - The result of methods that create sinks is now DataStreamSink instead
   of DataStream
 - Iterations cannot have feedback edges with differing parallelism
 - "Preserve partitioning" is not the default for feedback edges. The
   previous option for this is removed.
 - You can close an iteration several times, no need for a union.
 - Strict checking of whether partitioning and parallelism work
   together. I.e. if upstream and downstream parallelism don't match it
   is not legal to have Forward partitioning anymore. This was not very
   transparent: When you went from low parallelism to high dop some
   downstream  operators would never get any input. When you went from high
   parallelism to low dop you would get skew in the downstream operators
   because all elements that would be forwarded to an operator that is not
   "there" go to another operator. This requires insertion of global()
   or rebalance() in some places. For example with most sources which
   have parallelism one.

This also makes StreamExecutionEnvironment.execute() behave consistently
across different execution environments (local, remote ...): The list of
operators to be executed are cleared after execute is called.


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

Branch: refs/heads/master
Commit: bac21bf5d77c8e15c608ecbf006d29e7af1dd68a
Parents: baaa3da
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Jul 23 15:12:38 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Aug 19 17:53:04 2015 +0200

----------------------------------------------------------------------
 docs/apis/streaming_guide.md                    |  33 +-
 .../connectors/rabbitmq/RMQTopology.java        |   6 +-
 .../api/datastream/ConnectedDataStream.java     |  61 +-
 .../streaming/api/datastream/DataStream.java    | 604 +++++++++----------
 .../api/datastream/DataStreamSink.java          |  66 +-
 .../api/datastream/DataStreamSource.java        |  23 +-
 .../api/datastream/DiscretizedStream.java       |  17 +-
 .../api/datastream/GroupedDataStream.java       |   9 -
 .../api/datastream/IterativeDataStream.java     | 149 ++---
 .../api/datastream/KeyedDataStream.java         |  43 +-
 .../datastream/SingleOutputStreamOperator.java  | 109 ++--
 .../api/datastream/SplitDataStream.java         |  22 +-
 .../api/datastream/WindowedDataStream.java      |   6 +-
 .../temporal/StreamCrossOperator.java           |  43 +-
 .../datastream/temporal/StreamJoinOperator.java |  71 +--
 .../datastream/temporal/TemporalOperator.java   |   4 +-
 .../api/environment/LocalStreamEnvironment.java |   5 +-
 .../environment/RemoteStreamEnvironment.java    |   8 +-
 .../environment/StreamContextEnvironment.java   |   7 +-
 .../environment/StreamExecutionEnvironment.java | 178 ++++--
 .../api/environment/StreamPlanEnvironment.java  |   5 +-
 .../functions/source/FileSourceFunction.java    |   8 +
 .../streaming/api/graph/JSONGenerator.java      |  28 +-
 .../flink/streaming/api/graph/StreamConfig.java |  23 -
 .../flink/streaming/api/graph/StreamGraph.java  | 383 ++++++------
 .../api/graph/StreamGraphGenerator.java         | 530 ++++++++++++++++
 .../flink/streaming/api/graph/StreamLoop.java   | 122 ----
 .../flink/streaming/api/graph/StreamNode.java   |  27 +-
 .../api/graph/StreamingJobGraphGenerator.java   |  35 +-
 .../streaming/api/graph/WindowingOptimizer.java |  10 +-
 .../CoFeedbackTransformation.java               | 122 ++++
 .../transformations/FeedbackTransformation.java | 124 ++++
 .../transformations/OneInputTransformation.java | 117 ++++
 .../PartitionTransformation.java                |  80 +++
 .../transformations/SelectTransformation.java   |  83 +++
 .../api/transformations/SinkTransformation.java | 106 ++++
 .../transformations/SourceTransformation.java   |  70 +++
 .../transformations/SplitTransformation.java    |  84 +++
 .../transformations/StreamTransformation.java   | 320 ++++++++++
 .../transformations/TwoInputTransformation.java | 116 ++++
 .../transformations/UnionTransformation.java    |  81 +++
 .../partitioner/BroadcastPartitioner.java       |  17 +-
 .../partitioner/CustomPartitionerWrapper.java   |  11 +-
 .../runtime/partitioner/FieldsPartitioner.java  |  55 --
 .../runtime/partitioner/ForwardPartitioner.java |  46 ++
 .../runtime/partitioner/GlobalPartitioner.java  |  20 +-
 .../runtime/partitioner/HashPartitioner.java    |  63 ++
 .../partitioner/RebalancePartitioner.java       |  14 +-
 .../runtime/partitioner/ShufflePartitioner.java |  14 +-
 .../runtime/partitioner/StreamPartitioner.java  |  25 +-
 .../api/ChainedRuntimeContextTest.java          |  12 +-
 .../flink/streaming/api/CoStreamTest.java       |  10 +-
 .../flink/streaming/api/DataStreamTest.java     | 226 ++++---
 .../apache/flink/streaming/api/IterateTest.java | 459 +++++++-------
 .../flink/streaming/api/OutputSplitterTest.java |  11 +-
 .../flink/streaming/api/PartitionerTest.java    |  56 +-
 .../api/StreamExecutionEnvironmentTest.java     |  34 +-
 .../flink/streaming/api/TypeFillTest.java       |   3 +-
 .../streaming/api/WindowCrossJoinTest.java      |  10 +-
 .../api/collector/DirectedOutputTest.java       |   7 +-
 .../api/complex/ComplexIntegrationTest.java     |   3 -
 .../streaming/api/graph/SlotAllocationTest.java |   5 +-
 .../api/graph/StreamGraphGeneratorTest.java     | 179 ++++++
 .../graph/StreamingJobGraphGeneratorTest.java   |   9 +-
 .../api/operators/StreamProjectTest.java        |   8 +-
 .../api/operators/co/CoStreamFlatMapTest.java   |  19 +-
 .../api/operators/co/SelfConnectionTest.java    |  43 +-
 .../operators/windowing/WindowingITCase.java    |  62 +-
 .../api/outputformat/CsvOutputFormatITCase.java |   6 +-
 .../outputformat/TextOutputFormatITCase.java    |   6 +-
 .../api/state/StatefulOperatorTest.java         |   6 +-
 .../api/streamtask/StreamVertexTest.java        |  15 +-
 .../flink/streaming/graph/TranslationTest.java  |   6 +-
 .../partitioner/DistributePartitionerTest.java  |  56 --
 .../partitioner/FieldsPartitionerTest.java      |  71 ---
 .../partitioner/ForwardPartitionerTest.java     |   8 +-
 .../partitioner/HashPartitionerTest.java        |  71 +++
 .../partitioner/RebalancePartitionerTest.java   |  56 ++
 .../streaming/timestamp/TimestampITCase.java    |  21 +-
 .../streaming/util/EvenOddOutputSelector.java   |  31 +
 .../apache/flink/streaming/util/NoOpIntMap.java |  28 +
 .../apache/flink/streaming/util/NoOpSink.java   |  26 +
 .../streaming/util/ReceiveCheckNoOpSink.java    |  42 ++
 .../streaming/util/TestStreamEnvironment.java   |   9 +-
 .../flink/streaming/api/scala/DataStream.scala  |  24 +-
 .../api/scala/StreamCrossOperator.scala         |  10 +-
 .../api/scala/StreamJoinOperator.scala          |  26 +-
 .../streaming/api/scala/DataStreamTest.scala    | 195 +++---
 .../api/scala/OutputFormatTestPrograms.scala    |   4 +-
 .../streaming/api/scala/StateTestPrograms.scala |  12 +-
 .../StreamingScalaAPICompletenessTest.scala     |  11 +
 .../accumulators/AccumulatorLiveITCase.java     |  19 +-
 .../test/classloading/jar/StreamingProgram.java |   2 +-
 93 files changed, 4164 insertions(+), 1956 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/docs/apis/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md
index 35f6147..cd5acc3 100644
--- a/docs/apis/streaming_guide.md
+++ b/docs/apis/streaming_guide.md
@@ -290,25 +290,25 @@ instance (with its fields set to new values). The object reuse mode will lead to
 because fewer objects are created, but the user has to manually take care of what they are doing
 with the object references.
 
-### Partitioning
+### Data Shipping Strategies
 
-Partitioning controls how individual data points of a stream are distributed among the parallel instances of the transformation operators. This also controls the ordering of the records in the `DataStream`. There is partial ordering guarantee for the outputs with respect to the partitioning scheme (outputs produced from each partition are guaranteed to arrive in the order they were produced).
+The data shipping strategy controls how individual elements of a stream are distributed among the parallel instances of a transformation operator. This also controls the ordering of the records in the `DataStream`. There is partial ordering guarantee for the outputs with respect to the shipping strategy (outputs produced from each partition are guaranteed to arrive in the order they were produced).
 
-There are several partitioning types supported in Flink Streaming:
+These are the supported shipping strategies:
 
- * *Forward (default)*: Forward partitioning directs the output data to the next operator on the same machine (if possible) avoiding expensive network I/O. If there are more processing nodes than inputs or vice versa the load is distributed among the extra nodes in a round-robin fashion. This is the default partitioner.
+ * *Forward*: Forward shipping directs the output data to the next operator on the same machine, avoiding expensive network I/O. It can only be used when the parallelism of the input operations matches the parallelism of the downstream operation. This is the default shipping strategy if no strategy is specified and if the parallelism allows it.
 Usage: `dataStream.forward()`
- * *Shuffle*: Shuffle partitioning randomly partitions the output data stream to the next operator using uniform distribution. Use this only when it is important that the partitioning is randomised. If you only care about an even load use *Rebalance*.
+ * *Shuffle*: Shuffle randomly partitions the output data stream to the next operator using uniform distribution. Use this only when it is important that the partitioning is randomised. If you only care about an even load use *Rebalance*.
 Usage: `dataStream.shuffle()`
- * *Rebalance*: Rebalance partitioning directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution.
+ * *Rebalance*: Rebalance directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution. This is the default strategy if no strategy is defined and forward shipping is not possible because the parallelism of operations differs.
 Usage: `dataStream.rebalance()`
- * *Field/Key Partitioning*: Field/Key partitioning partitions the output data stream based on the hash code of a selected key of the tuples. Data points with the same key are directed to the same operator instance. 
+ * *Field/Key Partitioning*: Field/Key partitioning partitions the output data stream based on the hash code of a selected key of the tuples. Data points with the same key are directed to the same operator instance.
 Usage: `dataStream.partitionByHash(fields…)`
-* *Field/Key Grouping*: Field/Key grouping takes partitioning one step further and seperates the elements to disjoint groups based on the hash code. These groups are processed separately by the next downstream operator. 
+* *Field/Key Grouping*: Field/Key grouping takes field/key partitioning one step further and seperates the elements into disjoint groups based on the hash code. These groups are processed separately by the next downstream operator.
 Usage: `dataStream.groupBy(fields…)`
- * *Broadcast*: Broadcast partitioning sends the output data stream to all parallel instances of the next operator.
+ * *Broadcast*: Broadcast shipping sends the output data stream to all parallel instances of the next operator.
 Usage: `dataStream.broadcast()`
- * *Global*: All data points are directed to the first instance of the operator. 
+ * *Global*: All elements are directed to the first downstream instance of the operator.
 Usage: `dataStream.global()`
 
 Custom partitioning can also be used by giving a Partitioner function and a single field key to partition on, similarly to the batch API.
@@ -330,17 +330,15 @@ val result = in
 </div>
 </div>
 
-By default *Forward* partitioning is used. 
-
-Partitioning does not remain in effect after a transformation, so it needs to be set again for subsequent operations.
+The shipping strategy does not remain in effect after a transformation, so it needs to be set again for subsequent operations.
 
 ### Connecting to the outside world
 
-The user is expected to connect to the outside world through the source and the sink interfaces. 
+The user is expected to connect to the outside world through the source and the sink interfaces.
 
 #### Sources
 
-Sources can by created by using `StreamExecutionEnvironment.addSource(sourceFunction)`. 
+Sources can by created by using `StreamExecutionEnvironment.addSource(sourceFunction)`.
 Either use one of the source functions that come with Flink or write a custom source
 by implementing the `SourceFunction` interface. By default, sources run with
 parallelism of 1. To create parallel sources the user's source function needs to implement
@@ -534,7 +532,8 @@ dataStream.fold("", new FoldFunction<String, String>() {
     <tr>
       <td><strong>Union</strong></td>
       <td>
-        <p>Union of two or more data streams creating a new stream containing all the elements from all the streams.</p>
+        <p>Union of two or more data streams creating a new stream containing all the elements from all the streams. Node: If you union a data stream
+        with itself you will still only get each element once.</p>
 {% highlight java %}
 dataStream.union(otherStream1, otherStream2, …)
 {% endhighlight %}
@@ -1484,7 +1483,7 @@ Setting parallelism for operators works exactly the same way as in the batch Fli
 
 ### Buffer timeout
 
-By default, data points are not transferred on the network one-by-one, which would cause unnecessary network traffic, but are buffered in the output buffers. The size of the output buffers can be set in the Flink config files. While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough.
+By default, elements are not transferred on the network one-by-one, which would cause unnecessary network traffic, but are buffered in the output buffers. The size of the output buffers can be set in the Flink config files. While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough.
 To tackle this issue the user can call `env.setBufferTimeout(timeoutMillis)` on the execution environment (or on individual operators) to set a maximum wait time for the buffers to fill up. After this time, the buffers are flushed automatically even if they are not full. The default value for this timeout is 100 ms, which should be appropriate for most use-cases.
 
 Usage:

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index 0f06235..1f85862 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.connectors.rabbitmq;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
@@ -29,11 +29,11 @@ public class RMQTopology {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		@SuppressWarnings("unused")
-		DataStream<String> dataStream1 = env.addSource(
+		DataStreamSink<String> dataStream1 = env.addSource(
 				new RMQSource<String>("localhost", "hello", new SimpleStringSchema())).print();
 
 		@SuppressWarnings("unused")
-		DataStream<String> dataStream2 = env.fromElements("one", "two", "three", "four", "five",
+		DataStreamSink<String> dataStream2 = env.fromElements("one", "two", "three", "four", "five",
 				"q").addSink(
 				new RMQSink<String>("localhost", "hello", new StringToByteSerializer()));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index 079f136..8609a30 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -27,12 +27,12 @@ import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
 import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
-import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
 import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce;
 import org.apache.flink.streaming.api.operators.co.CoStreamMap;
 import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
 import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
 import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
 
@@ -49,7 +49,6 @@ import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
 public class ConnectedDataStream<IN1, IN2> {
 
 	protected StreamExecutionEnvironment environment;
-	protected StreamGraph jobGraphBuilder;
 	protected DataStream<IN1> dataStream1;
 	protected DataStream<IN2> dataStream2;
 
@@ -57,13 +56,13 @@ public class ConnectedDataStream<IN1, IN2> {
 	protected KeySelector<IN1, ?> keySelector1;
 	protected KeySelector<IN2, ?> keySelector2;
 
-	protected ConnectedDataStream(DataStream<IN1> input1, DataStream<IN2> input2) {
-		this.jobGraphBuilder = input1.streamGraph;
-		this.environment = input1.environment;
-		this.dataStream1 = input1.copy();
-		
+	protected ConnectedDataStream(StreamExecutionEnvironment env, DataStream<IN1> input1, DataStream<IN2> input2) {
+		this.environment = env;
+		if (input1 != null) {
+			this.dataStream1 = input1;
+		}
 		if (input2 != null) {
-			this.dataStream2 = input2.copy();
+			this.dataStream2 = input2;
 		}
 
 		if ((input1 instanceof GroupedDataStream) && (input2 instanceof GroupedDataStream)) {
@@ -78,7 +77,6 @@ public class ConnectedDataStream<IN1, IN2> {
 	}
 
 	protected ConnectedDataStream(ConnectedDataStream<IN1, IN2> coDataStream) {
-		this.jobGraphBuilder = coDataStream.jobGraphBuilder;
 		this.environment = coDataStream.environment;
 		this.dataStream1 = coDataStream.getFirst();
 		this.dataStream2 = coDataStream.getSecond();
@@ -105,7 +103,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The first DataStream.
 	 */
 	public DataStream<IN1> getFirst() {
-		return dataStream1.copy();
+		return dataStream1;
 	}
 
 	/**
@@ -114,7 +112,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The second DataStream.
 	 */
 	public DataStream<IN2> getSecond() {
-		return dataStream2.copy();
+		return dataStream2;
 	}
 
 	/**
@@ -150,7 +148,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The grouped {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.groupBy(keyPosition1),
+		return new ConnectedDataStream<IN1, IN2>(this.environment, dataStream1.groupBy(keyPosition1),
 				dataStream2.groupBy(keyPosition2));
 	}
 
@@ -167,7 +165,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The grouped {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> groupBy(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.groupBy(keyPositions1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(keyPositions1),
 				dataStream2.groupBy(keyPositions2));
 	}
 
@@ -185,7 +183,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The grouped {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> groupBy(String field1, String field2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.groupBy(field1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(field1),
 				dataStream2.groupBy(field2));
 	}
 
@@ -204,7 +202,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The grouped {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> groupBy(String[] fields1, String[] fields2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.groupBy(fields1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(fields1),
 				dataStream2.groupBy(fields2));
 	}
 
@@ -222,7 +220,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 */
 	public ConnectedDataStream<IN1, IN2> groupBy(KeySelector<IN1, ?> keySelector1,
 			KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.groupBy(keySelector1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(keySelector1),
 				dataStream2.groupBy(keySelector2));
 	}
 
@@ -239,7 +237,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The partitioned {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> partitionByHash(int keyPosition1, int keyPosition2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.partitionByHash(keyPosition1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keyPosition1),
 				dataStream2.partitionByHash(keyPosition2));
 	}
 
@@ -254,7 +252,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The partitioned {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> partitionByHash(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.partitionByHash(keyPositions1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keyPositions1),
 				dataStream2.partitionByHash(keyPositions2));
 	}
 
@@ -272,7 +270,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The partitioned {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> partitionByHash(String field1, String field2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.partitionByHash(field1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(field1),
 				dataStream2.partitionByHash(field2));
 	}
 
@@ -290,7 +288,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 * @return The partitioned {@link ConnectedDataStream}
 	 */
 	public ConnectedDataStream<IN1, IN2> partitionByHash(String[] fields1, String[] fields2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.partitionByHash(fields1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(fields1),
 				dataStream2.partitionByHash(fields2));
 	}
 
@@ -306,7 +304,7 @@ public class ConnectedDataStream<IN1, IN2> {
 	 */
 	public ConnectedDataStream<IN1, IN2> partitionByHash(KeySelector<IN1, ?> keySelector1,
 														KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedDataStream<IN1, IN2>(dataStream1.partitionByHash(keySelector1),
+		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keySelector1),
 				dataStream2.partitionByHash(keySelector2));
 	}
 
@@ -477,15 +475,22 @@ public class ConnectedDataStream<IN1, IN2> {
 	public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
 			TypeInformation<OUT> outTypeInfo, TwoInputStreamOperator<IN1, IN2, OUT> operator) {
 
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
-				environment, outTypeInfo, operator);
+		// read the output type of the input Transforms to coax out errors about MissinTypeInfo
+		dataStream1.getType();
+		dataStream2.getType();
 
-		dataStream1.streamGraph.addCoOperator(returnStream.getId(), operator, getType1(),
-				getType2(), outTypeInfo, functionName);
+		TwoInputTransformation<IN1, IN2, OUT> transform = new TwoInputTransformation<IN1, IN2, OUT>(
+				dataStream1.getTransformation(),
+				dataStream2.getTransformation(),
+				functionName,
+				operator,
+				outTypeInfo,
+				environment.getParallelism());
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(environment, transform);
 
-		dataStream1.connectGraph(dataStream1, returnStream.getId(), 1);
-		dataStream1.connectGraph(dataStream2, returnStream.getId(), 2);
+		getExecutionEnvironment().addOperator(transform);
 
 		return returnStream;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 10ed5e3..4f8ec18 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -17,13 +17,13 @@
 
 package org.apache.flink.streaming.api.datastream;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
+import com.google.common.collect.Lists;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.common.functions.RichFilterFunction;
@@ -39,7 +39,6 @@ import org.apache.flink.api.java.io.CsvOutputFormat;
 import org.apache.flink.api.java.io.TextOutputFormat;
 import org.apache.flink.api.java.operators.Keys;
 import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.MissingTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
@@ -51,12 +50,15 @@ import org.apache.flink.streaming.api.functions.sink.FileSinkFunctionByMillis;
 import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SocketClientSink;
-import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamFilter;
 import org.apache.flink.streaming.api.operators.StreamFlatMap;
 import org.apache.flink.streaming.api.operators.StreamMap;
 import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.api.transformations.UnionTransformation;
 import org.apache.flink.streaming.api.windowing.helper.Count;
 import org.apache.flink.streaming.api.windowing.helper.Delta;
 import org.apache.flink.streaming.api.windowing.helper.FullStream;
@@ -66,8 +68,9 @@ import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
 import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.FieldsPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
@@ -79,84 +82,29 @@ import com.google.common.base.Preconditions;
 /**
  * A DataStream represents a stream of elements of the same type. A DataStream
  * can be transformed into another DataStream by applying a transformation as
- * for example
+ * for example:
  * <ul>
- * <li>{@link DataStream#map},</li>
- * <li>{@link DataStream#filter}, or</li>
+ * <li>{@link DataStream#map},
+ * <li>{@link DataStream#filter}, or
  * </ul>
  * 
- * @param <OUT>
- *            The type of the DataStream, i.e., the type of the elements of the
- *            DataStream.
+ * @param <T> The type of the elements in this Stream
  */
-public class DataStream<OUT> {
+public class DataStream<T> {
 
-	protected static Integer counter = 0;
 	protected final StreamExecutionEnvironment environment;
-	protected final Integer id;
-	protected int parallelism;
-	protected List<String> selectedNames;
-	protected StreamPartitioner<OUT> partitioner;
-	@SuppressWarnings("rawtypes")
-	protected TypeInformation typeInfo;
-	protected List<DataStream<OUT>> unionedStreams;
-	
-	protected Integer iterationID = null;
-	protected Long iterationWaitTime = null;
 
-	protected final StreamGraph streamGraph;
-	private boolean typeUsed;
+	protected final StreamTransformation<T> transformation;
 
 	/**
 	 * Create a new {@link DataStream} in the given execution environment with
 	 * partitioning set to forward by default.
-	 * 
-	 * @param environment
-	 *            StreamExecutionEnvironment
-	 * @param typeInfo
-	 *            Type of the datastream
-	 */
-	public DataStream(StreamExecutionEnvironment environment, TypeInformation<OUT> typeInfo) {
-		if (environment == null) {
-			throw new NullPointerException("context is null");
-		}
-
-		counter++;
-		this.id = counter;
-		this.environment = environment;
-		this.parallelism = environment.getParallelism();
-		this.streamGraph = environment.getStreamGraph();
-		this.selectedNames = new ArrayList<String>();
-		this.partitioner = new RebalancePartitioner<OUT>(true);
-		this.typeInfo = typeInfo;
-		this.unionedStreams = new ArrayList<DataStream<OUT>>();
-		this.unionedStreams.add(this);
-	}
-
-	/**
-	 * Create a new DataStream by creating a copy of another DataStream
-	 * 
-	 * @param dataStream
-	 *            The DataStream that will be copied.
+	 *
+	 * @param environment The StreamExecutionEnvironment
 	 */
-	public DataStream(DataStream<OUT> dataStream) {
-		this.environment = dataStream.environment;
-		this.id = dataStream.id;
-		this.parallelism = dataStream.parallelism;
-		this.selectedNames = new ArrayList<String>(dataStream.selectedNames);
-		this.partitioner = dataStream.partitioner.copy();
-		this.streamGraph = dataStream.streamGraph;
-		this.typeInfo = dataStream.typeInfo;
-		this.iterationID = dataStream.iterationID;
-		this.iterationWaitTime = dataStream.iterationWaitTime;
-		this.unionedStreams = new ArrayList<DataStream<OUT>>();
-		this.unionedStreams.add(this);
-		
-		int size = dataStream.unionedStreams.size();
-		for (int i = 1; i < size; i++) {
-			this.unionedStreams.add(new DataStream<OUT>(dataStream.unionedStreams.get(i)));
-		}
-
+	public DataStream(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
+		this.environment = Preconditions.checkNotNull(environment, "Execution Environment must not be null.");
+		this.transformation = Preconditions.checkNotNull(transformation, "Stream Transformation must not be null.");
 	}
 
 	/**
@@ -165,7 +113,7 @@ public class DataStream<OUT> {
 	 * @return ID of the DataStream
 	 */
 	public Integer getId() {
-		return id;
+		return transformation.getId();
 	}
 
 	/**
@@ -174,15 +122,7 @@ public class DataStream<OUT> {
 	 * @return The parallelism set for this operator.
 	 */
 	public int getParallelism() {
-		return this.parallelism;
-	}
-	
-	public StreamPartitioner<OUT> getPartitioner() {
-		return this.partitioner;
-	}
-	
-	public List<String> getSelectedNames(){
-		return selectedNames;
+		return transformation.getParallelism();
 	}
 
 	/**
@@ -190,51 +130,26 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The type of the datastream.
 	 */
-	@SuppressWarnings("unchecked")
-	public TypeInformation<OUT> getType() {
-		if (typeInfo instanceof MissingTypeInfo) {
-			MissingTypeInfo typeInfo = (MissingTypeInfo) this.typeInfo;
-			throw new InvalidTypesException(
-					"The return type of function '"
-							+ typeInfo.getFunctionName()
-							+ "' could not be determined automatically, due to type erasure. "
-							+ "You can give type information hints by using the returns(...) method on the result of "
-							+ "the transformation call, or by letting your function implement the 'ResultTypeQueryable' "
-							+ "interface.", typeInfo.getTypeException());
-		}
-		typeUsed = true;
-		return this.typeInfo;
+	public TypeInformation<T> getType() {
+		return transformation.getOutputType();
 	}
 
 	/**
-	 * Tries to fill in the type information. Type information can be filled in
-	 * later when the program uses a type hint. This method checks whether the
-	 * type information has ever been accessed before and does not allow
-	 * modifications if the type was accessed already. This ensures consistency
-	 * by making sure different parts of the operation do not assume different
-	 * type information.
-	 * 
-	 * @param typeInfo
-	 *            The type information to fill in.
-	 * 
-	 * @throws IllegalStateException
-	 *             Thrown, if the type information has been accessed before.
+	 * Invokes the {@link org.apache.flink.api.java.ClosureCleaner}
+	 * on the given function if closure cleaning is enabled in the {@link ExecutionConfig}.
+	 *
+	 * @return The cleaned Function
 	 */
-	protected void fillInType(TypeInformation<OUT> typeInfo) {
-		if (typeUsed) {
-			throw new IllegalStateException(
-					"TypeInformation cannot be filled in for the type after it has been used. "
-							+ "Please make sure that the type info hints are the first call after the transformation function, "
-							+ "before any access to types or semantic properties, etc.");
-		}
-		streamGraph.setOutType(id, typeInfo);
-		this.typeInfo = typeInfo;
-	}
-
 	protected <F> F clean(F f) {
 		return getExecutionEnvironment().clean(f);
 	}
 
+	/**
+	 * Returns the {@link StreamExecutionEnvironment} that was used to create this
+	 * {@link DataStream}
+	 *
+	 * @return The Execution Environment
+	 */
 	public StreamExecutionEnvironment getExecutionEnvironment() {
 		return environment;
 	}
@@ -252,16 +167,27 @@ public class DataStream<OUT> {
 	 *            The DataStreams to union output with.
 	 * @return The {@link DataStream}.
 	 */
-	public DataStream<OUT> union(DataStream<OUT>... streams) {
-		DataStream<OUT> returnStream = this.copy();
+	public DataStream<T> union(DataStream<T>... streams) {
+		List<StreamTransformation<T>> unionedTransforms = Lists.newArrayList();
+		unionedTransforms.add(this.transformation);
+
+		Collection<StreamTransformation<?>> thisPredecessors = this.getTransformation().getTransitivePredecessors();
+
+		for (DataStream<T> newStream : streams) {
+			if (!(newStream.getParallelism() == this.getParallelism())) {
+				throw new UnsupportedClassVersionError(
+						"DataStream can only be unioned with DataStreams of the same parallelism. " +
+								"This Stream: " + this.getTransformation() +
+								", other stream: " + newStream.getTransformation());
+			}
+			Collection<StreamTransformation<?>> predecessors = newStream.getTransformation().getTransitivePredecessors();
 
-		for (DataStream<OUT> stream : streams) {
-			for (DataStream<OUT> ds : stream.unionedStreams) {
-				validateUnion(ds.getId());
-				returnStream.unionedStreams.add(ds.copy());
+			if (predecessors.contains(this.transformation) || thisPredecessors.contains(newStream.getTransformation())) {
+				throw new UnsupportedOperationException("A DataStream cannot be unioned with itself");
 			}
+			unionedTransforms.add(newStream.getTransformation());
 		}
-		return returnStream;
+		return new DataStream<T>(this.environment, new UnionTransformation<T>(unionedTransforms));
 	}
 
 	/**
@@ -275,12 +201,8 @@ public class DataStream<OUT> {
 	 *            for directing the tuples.
 	 * @return The {@link SplitDataStream}
 	 */
-	public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
-		for (DataStream<OUT> ds : this.unionedStreams) {
-			streamGraph.addOutputSelector(ds.getId(), clean(outputSelector));
-		}
-
-		return new SplitDataStream<OUT>(this);
+	public SplitDataStream<T> split(OutputSelector<T> outputSelector) {
+		return new SplitDataStream<T>(this, clean(outputSelector));
 	}
 
 	/**
@@ -293,8 +215,8 @@ public class DataStream<OUT> {
 	 *            The DataStream with which this stream will be connected.
 	 * @return The {@link ConnectedDataStream}.
 	 */
-	public <R> ConnectedDataStream<OUT, R> connect(DataStream<R> dataStream) {
-		return new ConnectedDataStream<OUT, R>(this, dataStream);
+	public <R> ConnectedDataStream<T, R> connect(DataStream<R> dataStream) {
+		return new ConnectedDataStream<T, R>(environment, this, dataStream);
 	}
 
 	/**
@@ -306,8 +228,8 @@ public class DataStream<OUT> {
 	 *            The KeySelector to be used for extracting the key for partitioning
 	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
 	 */
-	public KeyedDataStream<OUT> keyBy(KeySelector<OUT,?> key){
-		return new KeyedDataStream<OUT>(this, clean(key));
+	public KeyedDataStream<T> keyBy(KeySelector<T,?> key){
+		return new KeyedDataStream<T>(this, clean(key));
 	}
 
 	/**
@@ -318,11 +240,11 @@ public class DataStream<OUT> {
 	 *            will be grouped.
 	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
 	 */
-	public KeyedDataStream<OUT> keyBy(int... fields) {
+	public KeyedDataStream<T> keyBy(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return keyBy(new KeySelectorUtil.ArrayKeySelector<OUT>(fields));
+			return keyBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
 		} else {
-			return keyBy(new Keys.ExpressionKeys<OUT>(fields, getType()));
+			return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
 		}
 	}
 
@@ -337,12 +259,12 @@ public class DataStream<OUT> {
 	 *            partitioned.
 	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
 	 **/
-	public KeyedDataStream<OUT> keyBy(String... fields) {
-		return keyBy(new Keys.ExpressionKeys<OUT>(fields, getType()));
+	public KeyedDataStream<T> keyBy(String... fields) {
+		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
 	}
 
-	private KeyedDataStream<OUT> keyBy(Keys<OUT> keys) {
-		return new KeyedDataStream<OUT>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
+	private KeyedDataStream<T> keyBy(Keys<T> keys) {
+		return new KeyedDataStream<T>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
 				getType(), getExecutionConfig())));
 	}
 	
@@ -356,11 +278,11 @@ public class DataStream<OUT> {
 	 *            will be partitioned.
 	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
 	 */
-	public GroupedDataStream<OUT> groupBy(int... fields) {
+	public GroupedDataStream<T> groupBy(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return groupBy(new KeySelectorUtil.ArrayKeySelector<OUT>(fields));
+			return groupBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
 		} else {
-			return groupBy(new Keys.ExpressionKeys<OUT>(fields, getType()));
+			return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
 		}
 	}
 
@@ -369,7 +291,10 @@ public class DataStream<OUT> {
 	 * is either the name of a public field or a getter method with parentheses
 	 * of the {@link DataStream}S underlying type. A dot can be used to drill
 	 * down into objects, as in {@code "field1.getInnerField2()" }. This method
-	 * returns an {@link GroupedDataStream}.</p> This operator also affects the
+	 * returns an {@link GroupedDataStream}.
+	 *
+	 * <p>
+	 * This operator also affects the
 	 * partitioning of the stream, by forcing values with the same key to go to
 	 * the same processing instance.
 	 * 
@@ -378,14 +303,15 @@ public class DataStream<OUT> {
 	 *            grouped.
 	 * @return The grouped {@link DataStream}
 	 **/
-	public GroupedDataStream<OUT> groupBy(String... fields) {
-		return groupBy(new Keys.ExpressionKeys<OUT>(fields, getType()));
+	public GroupedDataStream<T> groupBy(String... fields) {
+		return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
 	}
 
 	/**
 	 * Groups the elements of a {@link DataStream} by the key extracted by the
 	 * {@link KeySelector} to be used with grouped operators like
-	 * {@link GroupedDataStream#reduce(ReduceFunction)}.
+	 * {@link GroupedDataStream#reduce(org.apache.flink.api.common.functions.ReduceFunction)}.
+	 *
 	 * <p/>
 	 * This operator also affects the partitioning of the stream, by forcing
 	 * values with the same key to go to the same processing instance.
@@ -395,12 +321,12 @@ public class DataStream<OUT> {
 	 *            the values
 	 * @return The grouped {@link DataStream}
 	 */
-	public GroupedDataStream<OUT> groupBy(KeySelector<OUT, ?> keySelector) {
-		return new GroupedDataStream<OUT>(this, clean(keySelector));
+	public GroupedDataStream<T> groupBy(KeySelector<T, ?> keySelector) {
+		return new GroupedDataStream<T>(this, clean(keySelector));
 	}
 
-	private GroupedDataStream<OUT> groupBy(Keys<OUT> keys) {
-		return new GroupedDataStream<OUT>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
+	private GroupedDataStream<T> groupBy(Keys<T> keys) {
+		return new GroupedDataStream<T>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
 				getType(), getExecutionConfig())));
 	}
 
@@ -412,14 +338,13 @@ public class DataStream<OUT> {
 	 *
 	 * @param fields The tuple fields that should be used for partitioning
 	 * @return The partitioned DataStream
-	 * Specifies how elements will be distributed to parallel instances of downstream operations.
 	 *
 	 */
-	public DataStream<OUT> partitionByHash(int... fields) {
+	public DataStream<T> partitionByHash(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return partitionByHash(new KeySelectorUtil.ArrayKeySelector<OUT>(fields));
+			return partitionByHash(new KeySelectorUtil.ArrayKeySelector<T>(fields));
 		} else {
-			return partitionByHash(new Keys.ExpressionKeys<OUT>(fields, getType()));
+			return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
 		}
 	}
 
@@ -431,11 +356,10 @@ public class DataStream<OUT> {
 	 *
 	 * @param fields The tuple fields that should be used for partitioning
 	 * @return The partitioned DataStream
-	 * Specifies how elements will be distributed to parallel instances of downstream operations.
 	 *
 	 */
-	public DataStream<OUT> partitionByHash(String... fields) {
-		return partitionByHash(new Keys.ExpressionKeys<OUT>(fields, getType()));
+	public DataStream<T> partitionByHash(String... fields) {
+		return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
 	}
 
 	/**
@@ -444,19 +368,21 @@ public class DataStream<OUT> {
 	 * effects the how the outputs will be distributed between the parallel
 	 * instances of the next processing operator.
 	 *
-	 * @param keySelector
+	 * @param keySelector The function that extracts the key from an element in the Stream
 	 * @return The partitioned DataStream
-	 * Specifies how elements will be distributed to parallel instances of downstream operations.
 	 */
-	public DataStream<OUT> partitionByHash(KeySelector<OUT, ?> keySelector) {
-		return setConnectionType(new FieldsPartitioner<OUT>(clean(keySelector)));
+	public DataStream<T> partitionByHash(KeySelector<T, ?> keySelector) {
+		return setConnectionType(new HashPartitioner<T>(clean(keySelector)));
 	}
 
 	//private helper method for partitioning
-	private DataStream<OUT> partitionByHash(Keys<OUT> keys) {
-		return setConnectionType(
-				new FieldsPartitioner<OUT>(
-						clean(KeySelectorUtil.getSelectorForKeys(keys, getType(), getExecutionConfig()))));
+	private DataStream<T> partitionByHash(Keys<T> keys) {
+		KeySelector<T, ?> keySelector = clean(KeySelectorUtil.getSelectorForKeys(
+				keys,
+				getType(),
+				getExecutionConfig()));
+
+		return setConnectionType(new HashPartitioner<T>(keySelector));
 	}
 
 	/**
@@ -469,8 +395,8 @@ public class DataStream<OUT> {
 	 * @param field The field index on which the DataStream is to partitioned.
 	 * @return The partitioned DataStream.
 	 */
-	public <K> DataStream<OUT> partitionCustom(Partitioner<K> partitioner, int field) {
-		Keys.ExpressionKeys<OUT> outExpressionKeys = new Keys.ExpressionKeys<OUT>(new int[]{field}, getType());
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, int field) {
+		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new int[]{field}, getType());
 		return partitionCustom(partitioner, outExpressionKeys);
 	}
 
@@ -484,8 +410,8 @@ public class DataStream<OUT> {
 	 * @param field The field index on which the DataStream is to partitioned.
 	 * @return The partitioned DataStream.
 	 */
-	public <K> DataStream<OUT> partitionCustom(Partitioner<K> partitioner, String field) {
-		Keys.ExpressionKeys<OUT> outExpressionKeys = new Keys.ExpressionKeys<OUT>(new String[]{field}, getType());
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, String field) {
+		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new String[]{field}, getType());
 		return partitionCustom(partitioner, outExpressionKeys);
 	}
 
@@ -505,16 +431,16 @@ public class DataStream<OUT> {
 	 * @return The partitioned DataStream.
 	 * @see KeySelector
 	 */
-	public <K> DataStream<OUT> partitionCustom(Partitioner<K> partitioner, KeySelector<OUT, K> keySelector) {
-		return setConnectionType(new CustomPartitionerWrapper<K, OUT>(clean(partitioner), clean(keySelector)));
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, KeySelector<T, K> keySelector) {
+		return setConnectionType(new CustomPartitionerWrapper<K, T>(clean(partitioner), clean(keySelector)));
 	}
 
 	//	private helper method for custom partitioning
-	private <K> DataStream<OUT> partitionCustom(Partitioner<K> partitioner, Keys<OUT> keys) {
-		KeySelector<OUT, K> keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig());
+	private <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, Keys<T> keys) {
+		KeySelector<T, K> keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig());
 
 		return setConnectionType(
-				new CustomPartitionerWrapper<K, OUT>(
+				new CustomPartitionerWrapper<K, T>(
 						clean(partitioner),
 						clean(keySelector)));
 	}
@@ -529,8 +455,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The DataStream with broadcast partitioning set.
 	 */
-	public DataStream<OUT> broadcast() {
-		return setConnectionType(new BroadcastPartitioner<OUT>());
+	public DataStream<T> broadcast() {
+		return setConnectionType(new BroadcastPartitioner<T>());
 	}
 
 	/**
@@ -543,8 +469,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The DataStream with shuffle partitioning set.
 	 */
-	public DataStream<OUT> shuffle() {
-		return setConnectionType(new ShufflePartitioner<OUT>());
+	public DataStream<T> shuffle() {
+		return setConnectionType(new ShufflePartitioner<T>());
 	}
 
 	/**
@@ -558,8 +484,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The DataStream with forward partitioning set.
 	 */
-	public DataStream<OUT> forward() {
-		return setConnectionType(new RebalancePartitioner<OUT>(true));
+	public DataStream<T> forward() {
+		return setConnectionType(new ForwardPartitioner<T>());
 	}
 
 	/**
@@ -573,8 +499,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The DataStream with rebalance partitioning set.
 	 */
-	public DataStream<OUT> rebalance() {
-		return setConnectionType(new RebalancePartitioner<OUT>(false));
+	public DataStream<T> rebalance() {
+		return setConnectionType(new RebalancePartitioner<T>());
 	}
 
 	/**
@@ -585,8 +511,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The DataStream with shuffle partitioning set.
 	 */
-	public DataStream<OUT> global() {
-		return setConnectionType(new GlobalPartitioner<OUT>());
+	public DataStream<T> global() {
+		return setConnectionType(new GlobalPartitioner<T>());
 	}
 
 	/**
@@ -607,7 +533,7 @@ public class DataStream<OUT> {
 	 * <p>
 	 * The iteration edge will be partitioned the same way as the first input of
 	 * the iteration head unless it is changed in the
-	 * {@link IterativeDataStream#closeWith(DataStream, boolean)} call.
+	 * {@link IterativeDataStream#closeWith(DataStream)} call.
 	 * <p>
 	 * By default a DataStream with iteration will never terminate, but the user
 	 * can use the maxWaitTime parameter to set a max waiting time for the
@@ -616,8 +542,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The iterative data stream created.
 	 */
-	public IterativeDataStream<OUT> iterate() {
-		return new IterativeDataStream<OUT>(this, 0);
+	public IterativeDataStream<T> iterate() {
+		return new IterativeDataStream<T>(this, 0);
 	}
 
 	/**
@@ -638,7 +564,7 @@ public class DataStream<OUT> {
 	 * <p>
 	 * The iteration edge will be partitioned the same way as the first input of
 	 * the iteration head unless it is changed in the
-	 * {@link IterativeDataStream#closeWith(DataStream, boolean)} call.
+	 * {@link IterativeDataStream#closeWith(DataStream)} call.
 	 * <p>
 	 * By default a DataStream with iteration will never terminate, but the user
 	 * can use the maxWaitTime parameter to set a max waiting time for the
@@ -651,8 +577,8 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The iterative data stream created.
 	 */
-	public IterativeDataStream<OUT> iterate(long maxWaitTimeMillis) {
-		return new IterativeDataStream<OUT>(this, maxWaitTimeMillis);
+	public IterativeDataStream<T> iterate(long maxWaitTimeMillis) {
+		return new IterativeDataStream<T>(this, maxWaitTimeMillis);
 	}
 
 	/**
@@ -669,12 +595,12 @@ public class DataStream<OUT> {
 	 *            output type
 	 * @return The transformed {@link DataStream}.
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> map(MapFunction<OUT, R> mapper) {
+	public <R> SingleOutputStreamOperator<R, ?> map(MapFunction<T, R> mapper) {
 
 		TypeInformation<R> outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(),
 				Utils.getCallLocationName(), true);
 
-		return transform("Map", outType, new StreamMap<OUT, R>(clean(mapper)));
+		return transform("Map", outType, new StreamMap<T, R>(clean(mapper)));
 	}
 
 	/**
@@ -693,12 +619,12 @@ public class DataStream<OUT> {
 	 *            output type
 	 * @return The transformed {@link DataStream}.
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> flatMap(FlatMapFunction<OUT, R> flatMapper) {
+	public <R> SingleOutputStreamOperator<R, ?> flatMap(FlatMapFunction<T, R> flatMapper) {
 
 		TypeInformation<R> outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper),
 				getType(), Utils.getCallLocationName(), true);
 
-		return transform("Flat Map", outType, new StreamFlatMap<OUT, R>(clean(flatMapper)));
+		return transform("Flat Map", outType, new StreamFlatMap<T, R>(clean(flatMapper)));
 
 	}
 
@@ -716,15 +642,17 @@ public class DataStream<OUT> {
 	 *            DataStream.
 	 * @return The filtered DataStream.
 	 */
-	public SingleOutputStreamOperator<OUT, ?> filter(FilterFunction<OUT> filter) {
-		return transform("Filter", getType(), new StreamFilter<OUT>(clean(filter)));
+	public SingleOutputStreamOperator<T, ?> filter(FilterFunction<T> filter) {
+		return transform("Filter", getType(), new StreamFilter<T>(clean(filter)));
 
 	}
 
 	/**
 	 * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.<br/>
-	 * <b>Note: Only Tuple DataStreams can be projected.</b></br> The
-	 * transformation projects each Tuple of the DataSet onto a (sub)set of
+	 * <b>Note: Only Tuple DataStreams can be projected.</b>
+	 *
+	 * <p>
+	 * The transformation projects each Tuple of the DataSet onto a (sub)set of
 	 * fields.
 	 * 
 	 * @param fieldIndexes
@@ -737,7 +665,7 @@ public class DataStream<OUT> {
 	 * @see DataStream
 	 */
 	public <R extends Tuple> SingleOutputStreamOperator<R, ?> project(int... fieldIndexes) {
-		return new StreamProjection<OUT>(this.copy(), fieldIndexes).projectTupleX();
+		return new StreamProjection<T>(this, fieldIndexes).projectTupleX();
 	}
 
 
@@ -762,15 +690,16 @@ public class DataStream<OUT> {
 	 *         cross transformation.
 	 * 
 	 */
-	public <IN2> StreamCrossOperator<OUT, IN2> cross(DataStream<IN2> dataStreamToCross) {
-		return new StreamCrossOperator<OUT, IN2>(this, dataStreamToCross);
+	public <IN2> StreamCrossOperator<T, IN2> cross(DataStream<IN2> dataStreamToCross) {
+		return new StreamCrossOperator<T, IN2>(this, dataStreamToCross);
 	}
 
 	/**
 	 * Initiates a temporal Join transformation. <br/>
 	 * A temporal Join transformation joins the elements of two
-	 * {@link DataStream}s on key equality over a specified time window.</br>
-	 * 
+	 * {@link DataStream}s on key equality over a specified time window.
+	 *
+	 * <p>
 	 * This method returns a {@link StreamJoinOperator} on which the
 	 * {@link StreamJoinOperator#onWindow(long, java.util.concurrent.TimeUnit)}
 	 * should be called to define the window, and then the
@@ -779,7 +708,7 @@ public class DataStream<OUT> {
 	 * the join keys.
 	 * <p>
 	 * The user can also use the
-	 * {@link StreamJoinOperator.JoinedStream#with(org.apache.flink.api.common.functions.JoinFunction)}
+	 * {@link org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator.JoinPredicate.JoinedStream#with}
 	 * to apply a custom join function.
 	 * 
 	 * @param dataStreamToJoin
@@ -788,8 +717,8 @@ public class DataStream<OUT> {
 	 *         Join transformation.
 	 * 
 	 */
-	public <IN2> StreamJoinOperator<OUT, IN2> join(DataStream<IN2> dataStreamToJoin) {
-		return new StreamJoinOperator<OUT, IN2>(this, dataStreamToJoin);
+	public <IN2> StreamJoinOperator<T, IN2> join(DataStream<IN2> dataStreamToJoin) {
+		return new StreamJoinOperator<T, IN2>(this, dataStreamToJoin);
 	}
 
 	/**
@@ -798,30 +727,46 @@ public class DataStream<OUT> {
 	 * {@link WindowedDataStream#mapWindow} or aggregations on preset
 	 * chunks(windows) of the data stream. To define windows a
 	 * {@link WindowingHelper} such as {@link Time}, {@link Count},
-	 * {@link Delta} and {@link FullStream} can be used.</br></br> When applied
-	 * to a grouped data stream, the windows (evictions) and slide sizes
-	 * (triggers) will be computed on a per group basis. </br></br> For more
-	 * advanced control over the trigger and eviction policies please refer to
-	 * {@link #window(TriggerPolicy, EvictionPolicy)} </br> </br> For example to create a
-	 * sum every 5 seconds in a tumbling fashion:</br>
-	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).sum(field)} </br></br> To
-	 * create sliding windows use the
-	 * {@link WindowedDataStream#every(WindowingHelper)} </br></br> The same
-	 * example with 3 second slides:</br>
+	 * {@link Delta} and {@link FullStream} can be used.
+	 *
+	 * <p>
+	 * When applied to a grouped data stream, the windows (evictions) and slide sizes
+	 * (triggers) will be computed on a per group basis.
+	 *
+	 * <p>
+	 * For more advanced control over the trigger and eviction policies please refer to
+	 * {@link #window(TriggerPolicy, EvictionPolicy)}
+	 *
+	 * <p>
+	 * For example, to create a sum every 5 seconds in a tumbling fashion:
+	 *
+	 * <pre>
+	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).sum(field)}
+	 * </pre>
+	 *
+	 * <p>
+	 * To create sliding windows use the
+	 * {@link WindowedDataStream#every(WindowingHelper)}, for example with 3 second slides:</br>
+	 *
+	 * <pre>
 	 * 
-	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).every(Time.of(3,
-	 *       TimeUnit.SECONDS)).sum(field)}
+	 * {@code
+	 * ds.window(Time.of(5, TimeUnit.SECONDS)).every(Time.of(3, TimeUnit.SECONDS)).sum(field)
+	 * }
+	 *
+	 * </pre>
 	 * 
 	 * @param policyHelper
 	 *            Any {@link WindowingHelper} such as {@link Time},
 	 *            {@link Count}, {@link Delta} {@link FullStream} to define the
 	 *            window size.
+	 *
 	 * @return A {@link WindowedDataStream} providing further operations.
 	 */
 	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public WindowedDataStream<OUT> window(WindowingHelper policyHelper) {
+	public WindowedDataStream<T> window(WindowingHelper policyHelper) {
 		policyHelper.setExecutionConfig(getExecutionConfig());
-		return new WindowedDataStream<OUT>(this, policyHelper);
+		return new WindowedDataStream<T>(this, policyHelper);
 	}
 
 	/**
@@ -829,8 +774,10 @@ public class DataStream<OUT> {
 	 * and {@link EvictionPolicy}. Windowing can be used to apply transformation
 	 * like {@link WindowedDataStream#reduceWindow},
 	 * {@link WindowedDataStream#mapWindow} or aggregations on preset
-	 * chunks(windows) of the data stream.</br></br>For most common use-cases
-	 * please refer to {@link #window(WindowingHelper)}
+	 * chunks(windows) of the data stream.
+	 *
+	 * <p>
+	 * For most common use-cases please refer to {@link #window(WindowingHelper)}
 	 * 
 	 * @param trigger
 	 *            The {@link TriggerPolicy} that will determine how often the
@@ -840,8 +787,8 @@ public class DataStream<OUT> {
 	 *            elements in each time window.
 	 * @return A {@link WindowedDataStream} providing further operations.
 	 */
-	public WindowedDataStream<OUT> window(TriggerPolicy<OUT> trigger, EvictionPolicy<OUT> eviction) {
-		return new WindowedDataStream<OUT>(this, trigger, eviction);
+	public WindowedDataStream<T> window(TriggerPolicy<T> trigger, EvictionPolicy<T> eviction) {
+		return new WindowedDataStream<T>(this, trigger, eviction);
 	}
 
 	/**
@@ -851,38 +798,44 @@ public class DataStream<OUT> {
 	 * @return A {@link WindowedDataStream} providing further operations.
 	 */
 	@SuppressWarnings("rawtypes")
-	public WindowedDataStream<OUT> every(WindowingHelper policyHelper) {
+	public WindowedDataStream<T> every(WindowingHelper policyHelper) {
 		policyHelper.setExecutionConfig(getExecutionConfig());
 		return window(FullStream.window()).every(policyHelper);
 	}
 
 	/**
-	 * Writes a DataStream to the standard output stream (stdout).<br>
+	 * Writes a DataStream to the standard output stream (stdout).
+	 *
+	 * <p>
 	 * For each element of the DataStream the result of
 	 * {@link Object#toString()} is written.
 	 * 
 	 * @return The closed DataStream.
 	 */
-	public DataStreamSink<OUT> print() {
-		PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>();
+	public DataStreamSink<T> print() {
+		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
 		return addSink(printFunction);
 	}
 
 	/**
-	 * Writes a DataStream to the standard output stream (stderr).<br>
+	 * Writes a DataStream to the standard output stream (stderr).
+	 *
+	 * <p>
 	 * For each element of the DataStream the result of
 	 * {@link Object#toString()} is written.
 	 * 
 	 * @return The closed DataStream.
 	 */
-	public DataStreamSink<OUT> printToErr() {
-		PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>(true);
+	public DataStreamSink<T> printToErr() {
+		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>(true);
 		return addSink(printFunction);
 	}
 
 	/**
-	 * Writes a DataStream to the file specified by path in text format. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
 	 * is written.
 	 * 
 	 * @param path
@@ -890,14 +843,16 @@ public class DataStream<OUT> {
 	 * 
 	 * @return the closed DataStream.
 	 */
-	public DataStreamSink<OUT> writeAsText(String path) {
-		return write(new TextOutputFormat<OUT>(new Path(path)), 0L);
+	public DataStreamSink<T> writeAsText(String path) {
+		return write(new TextOutputFormat<T>(new Path(path)), 0L);
 	}
 
 	/**
 	 * Writes a DataStream to the file specified by path in text format. The
-	 * writing is performed periodically, in every millis milliseconds. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
 	 * is written.
 	 * 
 	 * @param path
@@ -907,14 +862,16 @@ public class DataStream<OUT> {
 	 * 
 	 * @return the closed DataStream
 	 */
-	public DataStreamSink<OUT> writeAsText(String path, long millis) {
-		TextOutputFormat<OUT> tof = new TextOutputFormat<OUT>(new Path(path));
+	public DataStreamSink<T> writeAsText(String path, long millis) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
 		return write(tof, millis);
 	}
 
 	/**
-	 * Writes a DataStream to the file specified by path in text format. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
 	 * is written.
 	 * 
 	 * @param path
@@ -925,15 +882,17 @@ public class DataStream<OUT> {
 	 * 
 	 * @return the closed DataStream.
 	 */
-	public DataStreamSink<OUT> writeAsText(String path, WriteMode writeMode) {
-		TextOutputFormat<OUT> tof = new TextOutputFormat<OUT>(new Path(path));
+	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
 		tof.setWriteMode(writeMode);
 		return write(tof, 0L);
 	}
 
 	/**
-	 * Writes a DataStream to the file specified by path in text format. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
 	 * is written.
 	 * 
 	 * @param path
@@ -946,15 +905,17 @@ public class DataStream<OUT> {
 	 * 
 	 * @return the closed DataStream.
 	 */
-	public DataStreamSink<OUT> writeAsText(String path, WriteMode writeMode, long millis) {
-		TextOutputFormat<OUT> tof = new TextOutputFormat<OUT>(new Path(path));
+	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode, long millis) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
 		tof.setWriteMode(writeMode);
 		return write(tof, millis);
 	}
 
 	/**
-	 * Writes a DataStream to the file specified by path in csv format. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * Writes a DataStream to the file specified by path in csv format.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
 	 * is written. This method can only be used on data streams of tuples.
 	 * 
 	 * @param path
@@ -963,20 +924,22 @@ public class DataStream<OUT> {
 	 * @return the closed DataStream
 	 */
 	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<OUT> writeAsCsv(String path) {
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path) {
 		Preconditions.checkArgument(getType().isTupleType(),
 				"The writeAsCsv() method can only be used on data sets of tuples.");
 		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
 				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		return write((OutputFormat<OUT>) of, 0L);
+		return write((OutputFormat<T>) of, 0L);
 	}
 
 	/**
 	 * Writes a DataStream to the file specified by path in csv format. The
-	 * writing is performed periodically, in every millis milliseconds. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
 	 * is written. This method can only be used on data streams of tuples.
-	 * 
+	 *
 	 * @param path
 	 *            the path pointing to the location the text file is written to
 	 * @param millis
@@ -985,17 +948,19 @@ public class DataStream<OUT> {
 	 * @return the closed DataStream
 	 */
 	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<OUT> writeAsCsv(String path, long millis) {
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, long millis) {
 		Preconditions.checkArgument(getType().isTupleType(),
 				"The writeAsCsv() method can only be used on data sets of tuples.");
 		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
 				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		return write((OutputFormat<OUT>) of, millis);
+		return write((OutputFormat<T>) of, millis);
 	}
 
 	/**
-	 * Writes a DataStream to the file specified by path in csv format. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * Writes a DataStream to the file specified by path in csv format.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
 	 * is written. This method can only be used on data streams of tuples.
 	 * 
 	 * @param path
@@ -1007,7 +972,7 @@ public class DataStream<OUT> {
 	 * @return the closed DataStream
 	 */
 	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<OUT> writeAsCsv(String path, WriteMode writeMode) {
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode) {
 		Preconditions.checkArgument(getType().isTupleType(),
 				"The writeAsCsv() method can only be used on data sets of tuples.");
 		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
@@ -1015,13 +980,15 @@ public class DataStream<OUT> {
 		if (writeMode != null) {
 			of.setWriteMode(writeMode);
 		}
-		return write((OutputFormat<OUT>) of, 0L);
+		return write((OutputFormat<T>) of, 0L);
 	}
 
 	/**
 	 * Writes a DataStream to the file specified by path in csv format. The
-	 * writing is performed periodically, in every millis milliseconds. For
-	 * every element of the DataStream the result of {@link Object#toString()}
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
 	 * is written. This method can only be used on data streams of tuples.
 	 * 
 	 * @param path
@@ -1035,7 +1002,7 @@ public class DataStream<OUT> {
 	 * @return the closed DataStream
 	 */
 	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<OUT> writeAsCsv(String path, WriteMode writeMode,
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode,
 			long millis) {
 		Preconditions.checkArgument(getType().isTupleType(),
 				"The writeAsCsv() method can only be used on data sets of tuples.");
@@ -1044,7 +1011,7 @@ public class DataStream<OUT> {
 		if (writeMode != null) {
 			of.setWriteMode(writeMode);
 		}
-		return write((OutputFormat<OUT>) of, millis);
+		return write((OutputFormat<T>) of, millis);
 	}
 
 	/**
@@ -1059,8 +1026,8 @@ public class DataStream<OUT> {
 	 *            schema for serialization
 	 * @return the closed DataStream
 	 */
-	public DataStreamSink<OUT> writeToSocket(String hostName, int port, SerializationSchema<OUT, byte[]> schema) {
-		DataStreamSink<OUT> returnStream = addSink(new SocketClientSink<OUT>(hostName, port, schema));
+	public DataStreamSink<T> writeToSocket(String hostName, int port, SerializationSchema<T, byte[]> schema) {
+		DataStreamSink<T> returnStream = addSink(new SocketClientSink<T>(hostName, port, schema));
 		returnStream.setParallelism(1); // It would not work if multiple instances would connect to the same port
 		return returnStream;
 	}
@@ -1072,8 +1039,8 @@ public class DataStream<OUT> {
 	 * @param millis the write frequency
 	 * @return The closed DataStream
 	 */
-	public DataStreamSink<OUT> write(OutputFormat<OUT> format, long millis) {
-		return addSink(new FileSinkFunctionByMillis<OUT>(format, millis));
+	public DataStreamSink<T> write(OutputFormat<T> format, long millis) {
+		return addSink(new FileSinkFunctionByMillis<T>(format, millis));
 	}
 
 	/**
@@ -1090,66 +1057,35 @@ public class DataStream<OUT> {
 	 *            type of the return stream
 	 * @return the data stream constructed
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
-			TypeInformation<R> outTypeInfo, OneInputStreamOperator<OUT, R> operator) {
-		DataStream<OUT> inputStream = this.copy();
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
-				outTypeInfo, operator);
+	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName, TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
 
-		streamGraph.addOperator(returnStream.getId(), operator, getType(), outTypeInfo,
-				operatorName);
+		// read the output type of the input Transform to coax out errors about MissingTypeInfo
+		transformation.getOutputType();
 
-		connectGraph(inputStream, returnStream.getId(), 0);
-		
-		if (iterationID != null) {
-			//This data stream is an input to some iteration
-			addIterationSource(returnStream, null);
-		}
+		OneInputTransformation<T, R> resultTransform = new OneInputTransformation<T, R>(
+				this.transformation,
+				operatorName,
+				operator,
+				outTypeInfo,
+				environment.getParallelism());
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment, resultTransform);
+
+		getExecutionEnvironment().addOperator(resultTransform);
 
 		return returnStream;
 	}
-	
-	protected <X> void addIterationSource(DataStream<X> dataStream, TypeInformation<?> feedbackType) {
-		streamGraph.addIterationHead(dataStream.getId(), iterationID, iterationWaitTime, feedbackType);
-	}
 
 	/**
 	 * Internal function for setting the partitioner for the DataStream
-	 * 
+	 *
 	 * @param partitioner
 	 *            Partitioner to set.
 	 * @return The modified DataStream.
 	 */
-	protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
-		DataStream<OUT> returnStream = this.copy();
-
-		for (DataStream<OUT> stream : returnStream.unionedStreams) {
-			stream.partitioner = partitioner;
-		}
-
-		return returnStream;
-	}
-
-	/**
-	 * Internal function for assembling the underlying
-	 * {@link org.apache.flink.runtime.jobgraph.JobGraph} of the job. Connects
-	 * the outputs of the given input stream to the specified output stream
-	 * given by the outputID.
-	 * 
-	 * @param inputStream
-	 *            input data stream
-	 * @param outputID
-	 *            ID of the output
-	 * @param typeNumber
-	 *            Number of the type (used at co-functions)
-	 */
-	protected <X> void connectGraph(DataStream<X> inputStream, Integer outputID, int typeNumber) {
-		for (DataStream<X> stream : inputStream.unionedStreams) {
-			streamGraph.addEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
-					inputStream.selectedNames);
-		}
-
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		return new DataStream<T>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
 	}
 
 	/**
@@ -1161,24 +1097,26 @@ public class DataStream<OUT> {
 	 *            The object containing the sink's invoke function.
 	 * @return The closed DataStream.
 	 */
-	public DataStreamSink<OUT> addSink(SinkFunction<OUT> sinkFunction) {
-		return new DataStreamSink<OUT>((DataStream<OUT>) transform("StreamSink", null, new StreamSink<OUT>(clean(sinkFunction))));
-	}
+	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
 
-	private void validateUnion(Integer id) {
-		for (DataStream<OUT> ds : this.unionedStreams) {
-			if (ds.getId().equals(id)) {
-				throw new RuntimeException("A DataStream cannot be merged with itself");
-			}
-		}
+		// read the output type of the input Transform to coax out errors about MissingTypeInfo
+		transformation.getOutputType();
+
+		StreamSink<T> sinkOperator = new StreamSink<T>(clean(sinkFunction));
+
+		DataStreamSink<T> sink = new DataStreamSink<T>(this, sinkOperator);
+
+		getExecutionEnvironment().addOperator(sink.getTransformation());
+		return sink;
 	}
 
 	/**
-	 * Creates a copy of the {@link DataStream}
-	 * 
-	 * @return The copy
+	 * Returns the {@link StreamTransformation} that represents the operation that logically creates
+	 * this {@link DataStream}.
+	 *
+	 * @return The Transformation
 	 */
-	public DataStream<OUT> copy() {
-		return new DataStream<OUT>(this);
+	public StreamTransformation<T> getTransformation() {
+		return transformation;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index 60dc367..fdf398c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -17,30 +17,66 @@
 
 package org.apache.flink.streaming.api.datastream;
 
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.api.transformations.SinkTransformation;
 
 /**
- * Represents the end of a DataStream.
+ * A Stream Sink. This is used for emitting elements from a streaming topology.
  *
- * @param <IN>
- *            The type of the DataStream closed by the sink.
+ * @param <T> The type of the elements in the Stream
  */
-public class DataStreamSink<IN> extends SingleOutputStreamOperator<IN, DataStreamSink<IN>> {
+public class DataStreamSink<T> {
 
-	protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType,
-			TypeInformation<IN> outTypeInfo, OneInputStreamOperator<IN, ?> operator) {
-		super(environment, outTypeInfo, operator);
+	SinkTransformation<T> transformation;
+
+	@SuppressWarnings("unchecked")
+	protected DataStreamSink(DataStream<T> inputStream, StreamSink<T> operator) {
+		this.transformation = new SinkTransformation<T>(inputStream.getTransformation(), "Unnamed", operator, inputStream.getExecutionEnvironment().getParallelism());
+	}
+
+	/**
+	 * Returns the transformation that contains the actual sink operator of this sink.
+	 */
+	public SinkTransformation<T> getTransformation() {
+		return transformation;
 	}
 
-	protected DataStreamSink(DataStream<IN> dataStream) {
-		super(dataStream);
+	/**
+	 * Sets the name of this sink. This name is
+	 * used by the visualization and logging during runtime.
+	 *
+	 * @return The named sink.
+	 */
+	public DataStreamSink<T> name(String name) {
+		transformation.setName(name);
+		return this;
 	}
 
-	@Override
-	public DataStreamSink<IN> copy() {
-		throw new RuntimeException("Data stream sinks cannot be copied");
+	/**
+	 * Sets the parallelism for this sink. The degree must be higher than zero.
+	 *
+	 * @param parallelism The parallelism for this sink.
+	 * @return The sink with set parallelism.
+	 */
+	public DataStreamSink<T> setParallelism(int parallelism) {
+		transformation.setParallelism(parallelism);
+		return this;
 	}
 
+	/**
+	 * Turns off chaining for this operator so thread co-location will not be
+	 * used as an optimization.
+	 *
+	 * <p>
+	 * Chaining can be turned off for the whole
+	 * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()}
+	 * however it is not advised for performance considerations.
+	 *
+	 * @return The sink with chaining disabled
+	 */
+	public DataStreamSink<T> disableChaining() {
+		this.transformation.setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER);
+		return this;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
index 0dd7701..d2e04a7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
@@ -19,25 +19,22 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.transformations.SourceTransformation;
 
 /**
  * The DataStreamSource represents the starting point of a DataStream.
  * 
- * @param <OUT>
- *            Type of the DataStream created.
+ * @param <T> Type of the elements in the DataStream created from the this source.
  */
-public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataStreamSource<OUT>> {
+public class DataStreamSource<T> extends SingleOutputStreamOperator<T, DataStreamSource<T>> {
 
 	boolean isParallel;
 
-	public DataStreamSource(StreamExecutionEnvironment environment, String operatorType,
-			TypeInformation<OUT> outTypeInfo, StreamOperator<OUT> operator,
+	public DataStreamSource(StreamExecutionEnvironment environment,
+			TypeInformation<T> outTypeInfo, StreamSource<T> operator,
 			boolean isParallel, String sourceName) {
-		super(environment, outTypeInfo, operator);
-
-		environment.getStreamGraph().addSource(getId(), operator, null, outTypeInfo,
-				sourceName);
+		super(environment, new SourceTransformation<T>(sourceName, operator, outTypeInfo, environment.getParallelism()));
 
 		this.isParallel = isParallel;
 		if (!isParallel) {
@@ -46,11 +43,11 @@ public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataS
 	}
 
 	@Override
-	public DataStreamSource<OUT> setParallelism(int parallelism) {
+	public DataStreamSource<T> setParallelism(int parallelism) {
 		if (parallelism > 1 && !isParallel) {
-			throw new IllegalArgumentException("Source: " + this.id + " is not a parallel source");
+			throw new IllegalArgumentException("Source: " + transformation.getId() + " is not a parallel source");
 		} else {
-			return (DataStreamSource<OUT>) super.setParallelism(parallelism);
+			return (DataStreamSource<T>) super.setParallelism(parallelism);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
index e35592e..5893295 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
@@ -36,6 +36,7 @@ import org.apache.flink.streaming.api.operators.StreamFlatMap;
 import org.apache.flink.streaming.api.operators.windowing.EmptyWindowFilter;
 import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge;
 import org.apache.flink.streaming.api.operators.windowing.ParallelMerge;
+import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator;
 import org.apache.flink.streaming.api.operators.windowing.WindowFlattener;
 import org.apache.flink.streaming.api.operators.windowing.WindowFolder;
 import org.apache.flink.streaming.api.operators.windowing.WindowMapper;
@@ -43,7 +44,7 @@ import org.apache.flink.streaming.api.operators.windowing.WindowMerger;
 import org.apache.flink.streaming.api.operators.windowing.WindowPartExtractor;
 import org.apache.flink.streaming.api.operators.windowing.WindowPartitioner;
 import org.apache.flink.streaming.api.operators.windowing.WindowReducer;
-import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.windowing.StreamWindow;
 import org.apache.flink.streaming.api.windowing.StreamWindowTypeInfo;
 import org.apache.flink.streaming.api.windowing.WindowUtils.WindowKey;
@@ -96,7 +97,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 	}
 
 	public DataStream<OUT> flatten() {
-		return discretizedStream.transform("Window Flatten", getType(), new WindowFlattener<OUT>());
+		return discretizedStream.transform("Window Flatten", getType(), new WindowFlattener<OUT>()).setParallelism(discretizedStream.getParallelism());
 	}
 
 	public DataStream<StreamWindow<OUT>> getDiscretizedStream() {
@@ -112,7 +113,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 
 		// If we merged a non-grouped reduce transformation we need to reduce
 		// again
-		if (!isGrouped() && out.discretizedStream.operator instanceof WindowMerger) {
+		if (!isGrouped() && ((OneInputTransformation<?, ?>)out.discretizedStream.getTransformation()).getOperator() instanceof WindowMerger) {
 			return out.transform(WindowTransformation.REDUCEWINDOW, "Window Reduce", out.getType(),
 					new WindowReducer<OUT>(discretizedStream.clean(reduceFunction)));
 		} else {
@@ -171,7 +172,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 			TypeInformation<R> returnType) {
 		DiscretizedStream<R> out = partition(transformation).transform(
 				WindowTransformation.MAPWINDOW, "Window Map", returnType,
-				new WindowMapper<OUT, R>(discretizedStream.clean(windowMapFunction))).merge();
+				new WindowMapper<OUT, R>(discretizedStream.clean(windowMapFunction))).setParallelism(discretizedStream.getParallelism()).merge();
 
 		return out;
 	}
@@ -183,6 +184,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 		DiscretizedStream<R> out = partition(transformation).transform(
 				WindowTransformation.FOLDWINDOW, "Fold Window", outType,
 				new WindowFolder<OUT, R>(discretizedStream.clean(foldFunction), initialValue))
+				.setParallelism(discretizedStream.getParallelism())
 				.merge();
 		return out;
 	}
@@ -192,7 +194,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 			OneInputStreamOperator<StreamWindow<OUT>, StreamWindow<R>> operator) {
 
 		return wrap(discretizedStream.transform(operatorName, new StreamWindowTypeInfo<R>(retType),
-				operator), transformation);
+				operator).setParallelism(discretizedStream.getParallelism()), transformation);
 	}
 
 	private DiscretizedStream<OUT> filterEmpty(DiscretizedStream<OUT> input) {
@@ -246,7 +248,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 		if (isPartitioned) {
 			return wrap(
 					discretizedStream.groupBy(new WindowKey<OUT>()).transform("Window Merger",
-							type, new WindowMerger<OUT>()), false);
+							type, new WindowMerger<OUT>()).setParallelism(discretizedStream.getParallelism()), false);
 		} else {
 			return this;
 		}
@@ -327,8 +329,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 	}
 
 	protected DiscretizedStream<OUT> copy() {
-		return new DiscretizedStream<OUT>(discretizedStream.copy(), groupByKey, transformation,
-				isPartitioned);
+		return new DiscretizedStream<OUT>(discretizedStream, groupByKey, transformation, isPartitioned);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
index da005af..72ef945 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
@@ -52,9 +52,6 @@ public class GroupedDataStream<OUT> extends KeyedDataStream<OUT> {
 		super(dataStream, keySelector);
 	}
 
-	protected GroupedDataStream(GroupedDataStream<OUT> dataStream) {
-		super(dataStream);
-	}
 
 	/**
 	 * Applies a reduce transformation on the grouped data stream grouped on by
@@ -331,10 +328,4 @@ public class GroupedDataStream<OUT> extends KeyedDataStream<OUT> {
 				getType(), operator);
 		return returnStream;
 	}
-
-	@Override
-	public GroupedDataStream<OUT> copy() {
-		return new GroupedDataStream<OUT>(this);
-	}
-
 }


[5/6] flink git commit: [FLINK-2398][api-breaking] Introduce StreamGraphGenerator

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index 4de368c..6b12013 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -17,40 +17,41 @@
 
 package org.apache.flink.streaming.api.datastream;
 
-import java.util.List;
-
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+
+import java.util.Collection;
 
 /**
- * The iterative data stream represents the start of an iteration in a
- * {@link DataStream}.
+ * The iterative data stream represents the start of an iteration in a {@link DataStream}.
  * 
- * @param <IN>
- *            Type of the DataStream
+ * @param <T> Type of the elements in this Stream
  */
-public class IterativeDataStream<IN> extends
-		SingleOutputStreamOperator<IN, IterativeDataStream<IN>> {
-	
-	protected boolean closed = false;
+public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, IterativeDataStream<T>> {
 
-	static Integer iterationCount = 0;
+	// We store these so that we can create a co-iteration if we need to
+	private DataStream<T> originalInput;
+	private long maxWaitTime;
 	
-	protected IterativeDataStream(DataStream<IN> dataStream, long maxWaitTime) {
-		super(dataStream);
+	protected IterativeDataStream(DataStream<T> dataStream, long maxWaitTime) {
+		super(dataStream.getExecutionEnvironment(),
+				new FeedbackTransformation<T>(dataStream.getTransformation(), maxWaitTime));
+		this.originalInput = dataStream;
+		this.maxWaitTime = maxWaitTime;
 		setBufferTimeout(dataStream.environment.getBufferTimeout());
-		iterationID = iterationCount;
-		iterationCount++;
-		iterationWaitTime = maxWaitTime;
 	}
 
 	/**
 	 * Closes the iteration. This method defines the end of the iterative
-	 * program part that will be fed back to the start of the iteration. </br>
-	 * </br>A common usage pattern for streaming iterations is to use output
+	 * program part that will be fed back to the start of the iteration.
+	 *
+	 * <p>
+	 * A common usage pattern for streaming iterations is to use output
 	 * splitting to send a part of the closing data stream to the head. Refer to
 	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
 	 * for more information.
@@ -58,50 +59,30 @@ public class IterativeDataStream<IN> extends
 	 * @param feedbackStream
 	 *            {@link DataStream} that will be used as input to the iteration
 	 *            head.
-	 * @param keepPartitioning
-	 *            If true the feedback partitioning will be kept as it is (not
-	 *            changed to match the input of the iteration head)
+	 *
 	 * @return The feedback stream.
 	 * 
 	 */
 	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public DataStream<IN> closeWith(DataStream<IN> iterationTail, boolean keepPartitioning) {
-		
-		if (closed) {
-			throw new IllegalStateException(
-					"An iterative data stream can only be closed once. Use union to close with multiple stream.");
+	public DataStream<T> closeWith(DataStream<T> feedbackStream) {
+
+		Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+		if (!predecessors.contains(this.transformation)) {
+			throw new UnsupportedOperationException(
+					"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
 		}
-		closed = true;
-		
-		streamGraph.addIterationTail((List) iterationTail.unionedStreams, iterationID,
-				keepPartitioning);
 
-		return iterationTail;
-	}
-	
-	/**
-	 * Closes the iteration. This method defines the end of the iterative
-	 * program part that will be fed back to the start of the iteration. </br>
-	 * </br>A common usage pattern for streaming iterations is to use output
-	 * splitting to send a part of the closing data stream to the head. Refer to
-	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
-	 * for more information.
-	 * 
-	 * 
-	 * @param feedbackStream
-	 *            {@link DataStream} that will be used as input to the
-	 *            iteration head.
-	 * @return The feedback stream.
-	 * 
-	 */
-	public DataStream<IN> closeWith(DataStream<IN> iterationTail) {
-		return closeWith(iterationTail,false);
+		((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation());
+
+		return feedbackStream;
 	}
 
 	/**
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
 	 * {@link ConnectedDataStream}.
+	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
@@ -109,7 +90,7 @@ public class IterativeDataStream<IN> extends
 	 *            String describing the type information of the feedback stream.
 	 * @return A {@link ConnectedIterativeDataStream}.
 	 */
-	public <F> ConnectedIterativeDataStream<IN, F> withFeedbackType(String feedbackTypeString) {
+	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(String feedbackTypeString) {
 		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
 	}
 
@@ -117,6 +98,7 @@ public class IterativeDataStream<IN> extends
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
 	 * {@link ConnectedDataStream}.
+	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
@@ -124,7 +106,7 @@ public class IterativeDataStream<IN> extends
 	 *            Class of the elements in the feedback stream.
 	 * @return A {@link ConnectedIterativeDataStream}.
 	 */
-	public <F> ConnectedIterativeDataStream<IN, F> withFeedbackType(Class<F> feedbackTypeClass) {
+	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
 		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
 	}
 
@@ -132,6 +114,7 @@ public class IterativeDataStream<IN> extends
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
 	 * {@link ConnectedDataStream}.
+	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
@@ -139,9 +122,8 @@ public class IterativeDataStream<IN> extends
 	 *            The type information of the feedback stream.
 	 * @return A {@link ConnectedIterativeDataStream}.
 	 */
-	public <F> ConnectedIterativeDataStream<IN, F> withFeedbackType(TypeInformation<F> feedbackType) {
-		return new ConnectedIterativeDataStream<IN, F>(new IterativeDataStream<IN>(this,
-				iterationWaitTime), feedbackType);
+	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
+		return new ConnectedIterativeDataStream<T, F>(originalInput, feedbackType, maxWaitTime);
 	}
 	
 	/**
@@ -149,6 +131,7 @@ public class IterativeDataStream<IN> extends
 	 * iterative part of a streaming program, where the original input of the
 	 * iteration and the feedback of the iteration are connected as in a
 	 * {@link ConnectedDataStream}.
+	 *
 	 * <p>
 	 * The user can distinguish between the two inputs using co-transformation,
 	 * thus eliminating the need for mapping the inputs and outputs to a common
@@ -161,38 +144,18 @@ public class IterativeDataStream<IN> extends
 	 */
 	public static class ConnectedIterativeDataStream<I, F> extends ConnectedDataStream<I, F>{
 
-		private IterativeDataStream<I> input;
-		private TypeInformation<F> feedbackType;
+		private CoFeedbackTransformation<F> coFeedbackTransformation;
 
-		public ConnectedIterativeDataStream(IterativeDataStream<I> input, TypeInformation<F> feedbackType) {
-			super(input, null);
-			this.input = input;
-			this.feedbackType = feedbackType;
+		public ConnectedIterativeDataStream(DataStream<I> input, TypeInformation<F> feedbackType, long waitTime) {
+			super(input.getExecutionEnvironment(),
+					input,
+					new DataStream<F>(input.getExecutionEnvironment(),
+							new CoFeedbackTransformation<F>(input.getParallelism(),
+									feedbackType,
+									waitTime)));
+			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecond().getTransformation();
 		}
-		
-		@Override
-		public TypeInformation<F> getType2() {
-			return feedbackType;
-		}
-		
-		@Override
-		public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
-				TypeInformation<OUT> outTypeInfo, TwoInputStreamOperator<I, F, OUT> operator) {
 
-			@SuppressWarnings({ "unchecked", "rawtypes" })
-			SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
-					input.environment, outTypeInfo, operator);
-
-			input.streamGraph.addCoOperator(returnStream.getId(), operator, input.getType(),
-					feedbackType, outTypeInfo, functionName);
-
-			input.connectGraph(input, returnStream.getId(), 1);
-			
-			input.addIterationSource(returnStream, feedbackType);
-
-			return returnStream;
-		}
-		
 		/**
 		 * Closes the iteration. This method defines the end of the iterative
 		 * program part that will be fed back to the start of the iteration as
@@ -206,14 +169,16 @@ public class IterativeDataStream<IN> extends
 		 */
 		@SuppressWarnings({ "rawtypes", "unchecked" })
 		public DataStream<F> closeWith(DataStream<F> feedbackStream) {
-			if (input.closed) {
-				throw new IllegalStateException(
-						"An iterative data stream can only be closed once. Use union to close with multiple stream.");
+
+			Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+			if (!predecessors.contains(this.coFeedbackTransformation)) {
+				throw new UnsupportedOperationException(
+						"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
 			}
-			input.closed = true;
-			
-			input.streamGraph.addIterationTail((List) feedbackStream.unionedStreams,
-					input.iterationID, true);
+
+			coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation());
+
 			return feedbackStream;
 		}
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
index b944302..7628815 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
@@ -19,7 +19,11 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 
 /**
@@ -28,11 +32,10 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
  * are also possible on a KeyedDataStream, with the exception of partitioning methods such as shuffle, forward and groupBy.
  * 
  * 
- * @param <OUT>
- *            The output type of the {@link KeyedDataStream}.
+ * @param <T> The type of the elements in the Keyed Stream
  */
-public class KeyedDataStream<OUT> extends DataStream<OUT> {
-	KeySelector<OUT, ?> keySelector;
+public class KeyedDataStream<T> extends DataStream<T> {
+	KeySelector<T, ?> keySelector;
 
 	/**
 	 * Creates a new {@link KeyedDataStream} using the given {@link KeySelector}
@@ -43,35 +46,35 @@ public class KeyedDataStream<OUT> extends DataStream<OUT> {
 	 * @param keySelector
 	 *            Function for determining state partitions
 	 */
-	public KeyedDataStream(DataStream<OUT> dataStream, KeySelector<OUT, ?> keySelector) {
-		super(dataStream.partitionByHash(keySelector));
+	public KeyedDataStream(DataStream<T> dataStream, KeySelector<T, ?> keySelector) {
+		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<T>(dataStream.getTransformation(), new HashPartitioner<T>(keySelector)));
 		this.keySelector = keySelector;
 	}
 
-	protected KeyedDataStream(KeyedDataStream<OUT> dataStream) {
-		super(dataStream);
-		this.keySelector = dataStream.keySelector;
-	}
-
-	public KeySelector<OUT, ?> getKeySelector() {
+	public KeySelector<T, ?> getKeySelector() {
 		return this.keySelector;
 	}
 
 	@Override
-	protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
 		throw new UnsupportedOperationException("Cannot override partitioning for KeyedDataStream.");
 	}
 
 	@Override
-	public KeyedDataStream<OUT> copy() {
-		return new KeyedDataStream<OUT>(this);
-	}
-
-	@Override
 	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
-			TypeInformation<R> outTypeInfo, OneInputStreamOperator<OUT, R> operator) {
+			TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
+
 		SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
-		streamGraph.setKey(returnStream.getId(), keySelector);
+
+		((OneInputTransformation<T, R>) returnStream.getTransformation()).setStateKeySelector(
+				keySelector);
 		return returnStream;
 	}
+
+	@Override
+	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
+		DataStreamSink<T> result = super.addSink(sinkFunction);
+		result.getTransformation().setStateKeySelector(keySelector);
+		return result;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index b4a99c8..016cf5e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -24,23 +24,23 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 
 /**
  * The SingleOutputStreamOperator represents a user defined transformation
  * applied on a {@link DataStream} with one predefined output type.
- * 
- * @param <OUT>
- *            Output type of the operator.
- * @param <O>
- *            Type of the operator.
+ *
+ * @param <T> The type of the elements in this Stream
+ * @param <O> Type of the operator.
  */
-public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends
-		DataStream<OUT> {
+public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<T, O>> extends DataStream<T> {
 
-	protected boolean isSplit;
-	protected StreamOperator<?> operator;
+	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
+		super(environment, transformation);
+	}
 
 	/**
 	 * Gets the name of the current data stream. This name is
@@ -48,8 +48,8 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *
 	 * @return Name of the stream.
 	 */
-	public String getName(){
-		return streamGraph.getStreamNode(getId()).getOperatorName();
+	public String getName() {
+		return transformation.getName();
 	}
 
 	/**
@@ -58,27 +58,11 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *
 	 * @return The named operator.
 	 */
-	public DataStream<OUT> name(String name){
-		streamGraph.getStreamNode(id).setOperatorName(name);
+	public SingleOutputStreamOperator<T, O> name(String name){
+		transformation.setName(name);
 		return this;
 	}
 
-	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment,
-			TypeInformation<OUT> outTypeInfo, StreamOperator<?> operator) {
-		super(environment, outTypeInfo);
-		this.isSplit = false;
-		this.operator = operator;
-	}
-
-	@SuppressWarnings("unchecked")
-	protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
-		super(dataStream);
-		if (dataStream instanceof SingleOutputStreamOperator) {
-			this.isSplit = ((SingleOutputStreamOperator<OUT, ?>) dataStream).isSplit;
-			this.operator = ((SingleOutputStreamOperator<OUT, ?>) dataStream).operator;
-		}
-	}
-
 	/**
 	 * Sets the parallelism for this operator. The degree must be 1 or more.
 	 * 
@@ -86,13 +70,12 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *            The parallelism for this operator.
 	 * @return The operator with set parallelism.
 	 */
-	public SingleOutputStreamOperator<OUT, O> setParallelism(int parallelism) {
+	public SingleOutputStreamOperator<T, O> setParallelism(int parallelism) {
 		if (parallelism < 1) {
 			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
 		}
-		this.parallelism = parallelism;
 
-		streamGraph.setParallelism(id, parallelism);
+		transformation.setParallelism(parallelism);
 
 		return this;
 	}
@@ -105,39 +88,34 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *            The maximum time between two output flushes.
 	 * @return The operator with buffer timeout set.
 	 */
-	public SingleOutputStreamOperator<OUT, O> setBufferTimeout(long timeoutMillis) {
-		streamGraph.setBufferTimeout(id, timeoutMillis);
+	public SingleOutputStreamOperator<T, O> setBufferTimeout(long timeoutMillis) {
+		transformation.setBufferTimeout(timeoutMillis);
 		return this;
 	}
 
 	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> broadcast() {
-		return (SingleOutputStreamOperator<OUT, O>) super.broadcast();
+	public SingleOutputStreamOperator<T, O> broadcast() {
+		return (SingleOutputStreamOperator<T, O>) super.broadcast();
 	}
 
 	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> shuffle() {
-		return (SingleOutputStreamOperator<OUT, O>) super.shuffle();
+	public SingleOutputStreamOperator<T, O> shuffle() {
+		return (SingleOutputStreamOperator<T, O>) super.shuffle();
 	}
 
 	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> forward() {
-		return (SingleOutputStreamOperator<OUT, O>) super.forward();
+	public SingleOutputStreamOperator<T, O> forward() {
+		return (SingleOutputStreamOperator<T, O>) super.forward();
 	}
 
 	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> rebalance() {
-		return (SingleOutputStreamOperator<OUT, O>) super.rebalance();
+	public SingleOutputStreamOperator<T, O> rebalance() {
+		return (SingleOutputStreamOperator<T, O>) super.rebalance();
 	}
 
 	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> global() {
-		return (SingleOutputStreamOperator<OUT, O>) super.global();
-	}
-
-	@Override
-	public SingleOutputStreamOperator<OUT, O> copy() {
-		return new SingleOutputStreamOperator<OUT, O>(this);
+	public SingleOutputStreamOperator<T, O> global() {
+		return (SingleOutputStreamOperator<T, O>) super.global();
 	}
 
 	/**
@@ -149,8 +127,8 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *            The selected {@link ChainingStrategy}
 	 * @return The operator with the modified chaining strategy
 	 */
-	private SingleOutputStreamOperator<OUT, O> setChainingStrategy(ChainingStrategy strategy) {
-		this.operator.setChainingStrategy(strategy);
+	private SingleOutputStreamOperator<T, O> setChainingStrategy(ChainingStrategy strategy) {
+		this.transformation.setChainingStrategy(strategy);
 		return this;
 	}
 
@@ -162,7 +140,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * 
 	 * @return The operator with chaining disabled
 	 */
-	public SingleOutputStreamOperator<OUT, O> disableChaining() {
+	public SingleOutputStreamOperator<T, O> disableChaining() {
 		return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER);
 	}
 
@@ -173,7 +151,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * 
 	 * @return The operator with chaining set.
 	 */
-	public SingleOutputStreamOperator<OUT, O> startNewChain() {
+	public SingleOutputStreamOperator<T, O> startNewChain() {
 		return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.HEAD);
 	}
 
@@ -216,7 +194,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 		if (typeInfoString == null) {
 			throw new IllegalArgumentException("Type information string must not be null.");
 		}
-		return returns(TypeInfoParser.<OUT>parse(typeInfoString));
+		return returns(TypeInfoParser.<T>parse(typeInfoString));
 	}
 	
 	/**
@@ -243,11 +221,11 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 *            type information as a return type hint
 	 * @return This operator with a given return type hint.
 	 */
-	public O returns(TypeInformation<OUT> typeInfo) {
+	public O returns(TypeInformation<T> typeInfo) {
 		if (typeInfo == null) {
 			throw new IllegalArgumentException("Type information must not be null.");
 		}
-		fillInType(typeInfo);
+		transformation.setOutputType(typeInfo);
 		@SuppressWarnings("unchecked")
 		O returnType = (O) this;
 		return returnType;
@@ -277,13 +255,13 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * @return This operator with a given return type hint.
 	 */
 	@SuppressWarnings("unchecked")
-	public O returns(Class<OUT> typeClass) {
+	public O returns(Class<T> typeClass) {
 		if (typeClass == null) {
 			throw new IllegalArgumentException("Type class must not be null.");
 		}
 		
 		try {
-			TypeInformation<OUT> ti = (TypeInformation<OUT>) TypeExtractor.createTypeInfo(typeClass);
+			TypeInformation<T> ti = (TypeInformation<T>) TypeExtractor.createTypeInfo(typeClass);
 			return returns(ti);
 		}
 		catch (InvalidTypesException e) {
@@ -291,6 +269,11 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 		}
 	}
 
+	@Override
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		return new SingleOutputStreamOperator<T, O>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
+	}
+
 	/**
 	 * By default all operators in a streaming job share the same resource
 	 * group. Each resource group takes as many task manager slots as the
@@ -305,8 +288,8 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * 
 	 * @return The operator as a part of a new resource group.
 	 */
-	public SingleOutputStreamOperator<OUT, O> startNewResourceGroup() {
-		streamGraph.setResourceStrategy(getId(), ResourceStrategy.NEWGROUP);
+	public SingleOutputStreamOperator<T, O> startNewResourceGroup() {
+		transformation.setResourceStrategy(ResourceStrategy.NEWGROUP);
 		return this;
 	}
 
@@ -319,8 +302,8 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * 
 	 * @return The operator with isolated resource group.
 	 */
-	public SingleOutputStreamOperator<OUT, O> isolateResources() {
-		streamGraph.setResourceStrategy(getId(), ResourceStrategy.ISOLATE);
+	public SingleOutputStreamOperator<T, O> isolateResources() {
+		transformation.setResourceStrategy(ResourceStrategy.ISOLATE);
 		return this;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
index 6b95fe7..bc9ecfb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
@@ -17,23 +17,23 @@
 
 package org.apache.flink.streaming.api.datastream;
 
-import java.util.Arrays;
-
+import com.google.common.collect.Lists;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.transformations.SelectTransformation;
+import org.apache.flink.streaming.api.transformations.SplitTransformation;
 
 /**
  * The SplitDataStream represents an operator that has been split using an
  * {@link OutputSelector}. Named outputs can be selected using the
  * {@link #select} function. To apply transformation on the whole output simply
  * call the transformation on the SplitDataStream
- * 
- * @param <OUT>
- *            The type of the output.
+ *
+ * @param <OUT> The type of the elements in the Stream
  */
 public class SplitDataStream<OUT> extends DataStream<OUT> {
 
-	protected SplitDataStream(DataStream<OUT> dataStream) {
-		super(dataStream);
+	protected SplitDataStream(DataStream<OUT> dataStream, OutputSelector<OUT> outputSelector) {
+		super(dataStream.getExecutionEnvironment(), new SplitTransformation<OUT>(dataStream.getTransformation(), outputSelector));
 	}
 
 	/**
@@ -55,12 +55,8 @@ public class SplitDataStream<OUT> extends DataStream<OUT> {
 			}
 		}
 
-		DataStream<OUT> returnStream = copy();
-
-		for (DataStream<OUT> ds : returnStream.unionedStreams) {
-			ds.selectedNames = Arrays.asList(outputNames);
-		}
-		return returnStream;
+		SelectTransformation<OUT> selectTransform = new SelectTransformation<OUT>(this.getTransformation(), Lists.newArrayList(outputNames));
+		return new DataStream<OUT>(this.getExecutionEnvironment(), selectTransform);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
index 9565f4b..bf3a11a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
@@ -99,7 +99,7 @@ public class WindowedDataStream<OUT> {
 	protected EvictionPolicy<OUT> userEvicter;
 
 	protected WindowedDataStream(DataStream<OUT> dataStream, WindowingHelper<OUT> policyHelper) {
-		this.dataStream = dataStream.copy();
+		this.dataStream = dataStream;
 		this.triggerHelper = policyHelper;
 
 		if (dataStream instanceof GroupedDataStream) {
@@ -109,7 +109,7 @@ public class WindowedDataStream<OUT> {
 
 	protected WindowedDataStream(DataStream<OUT> dataStream, TriggerPolicy<OUT> trigger,
 			EvictionPolicy<OUT> evicter) {
-		this.dataStream = dataStream.copy();
+		this.dataStream = dataStream;
 
 		this.userTrigger = trigger;
 		this.userEvicter = evicter;
@@ -120,7 +120,7 @@ public class WindowedDataStream<OUT> {
 	}
 
 	protected WindowedDataStream(WindowedDataStream<OUT> windowedDataStream) {
-		this.dataStream = windowedDataStream.dataStream.copy();
+		this.dataStream = windowedDataStream.dataStream;
 		this.discretizerKey = windowedDataStream.discretizerKey;
 		this.groupByKey = windowedDataStream.groupByKey;
 		this.triggerHelper = windowedDataStream.triggerHelper;

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
index 3dd02a3..e0aafb7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
@@ -30,10 +30,9 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.functions.co.CrossWindowFunction;
-import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
 
 public class StreamCrossOperator<I1, I2> extends
-		TemporalOperator<I1, I2, StreamCrossOperator.CrossWindow<I1, I2>> {
+		TemporalOperator<I1, I2, StreamCrossOperator.CrossWindow<I1, I2, Tuple2<I1, I2>>> {
 
 	public StreamCrossOperator(DataStream<I1> input1, DataStream<I2> input2) {
 		super(input1, input2);
@@ -48,37 +47,42 @@ public class StreamCrossOperator<I1, I2> extends
 	}
 
 	@Override
-	protected CrossWindow<I1, I2> createNextWindowOperator() {
+	protected CrossWindow<I1, I2, Tuple2<I1, I2>> createNextWindowOperator() {
 
 		CrossWindowFunction<I1, I2, Tuple2<I1, I2>> crossWindowFunction = new CrossWindowFunction<I1, I2, Tuple2<I1, I2>>(
 				clean(new CrossOperator.DefaultCrossFunction<I1, I2>()));
 
-		return new CrossWindow<I1, I2>(this, input1.connect(input2).addGeneralWindowCombine(
+		return new CrossWindow<I1, I2, Tuple2<I1, I2>>(this, input1.connect(input2).addGeneralWindowCombine(
 				crossWindowFunction,
 				new TupleTypeInfo<Tuple2<I1, I2>>(input1.getType(), input2.getType()), windowSize,
 				slideInterval, timeStamp1, timeStamp2));
 	}
 
-	public static class CrossWindow<I1, I2> extends
-			SingleOutputStreamOperator<Tuple2<I1, I2>, CrossWindow<I1, I2>> implements
-			TemporalWindow<CrossWindow<I1, I2>> {
+	public static class CrossWindow<I1, I2, R> extends
+			SingleOutputStreamOperator<R, CrossWindow<I1, I2, R>> implements
+			TemporalWindow<CrossWindow<I1, I2, R>> {
 
 		private StreamCrossOperator<I1, I2> op;
 
-		public CrossWindow(StreamCrossOperator<I1, I2> op, DataStream<Tuple2<I1, I2>> ds) {
-			super(ds);
+		public CrossWindow(StreamCrossOperator<I1, I2> op, DataStream<R> ds) {
+			super(ds.getExecutionEnvironment(), ds.getTransformation());
 			this.op = op;
 		}
 
-		public CrossWindow<I1, I2> every(long length, TimeUnit timeUnit) {
+		public CrossWindow<I1, I2, R> every(long length, TimeUnit timeUnit) {
 			return every(timeUnit.toMillis(length));
 		}
 
 		@SuppressWarnings("unchecked")
-		public CrossWindow<I1, I2> every(long length) {
-			((CoStreamWindow<I1, I2, ?>) streamGraph.getStreamNode(id).getOperator())
-					.setSlideSize(length);
-			return this;
+		public CrossWindow<I1, I2, R> every(long length) {
+
+			CrossWindowFunction<I1, I2, Tuple2<I1, I2>> crossWindowFunction = new CrossWindowFunction<I1, I2, Tuple2<I1, I2>>(
+					clean(new CrossOperator.DefaultCrossFunction<I1, I2>()));
+
+			return (CrossWindow<I1, I2, R>) new CrossWindow<I1, I2, Tuple2<I1, I2>>(op, op.input1.connect(op.input2).addGeneralWindowCombine(
+					crossWindowFunction,
+					new TupleTypeInfo<Tuple2<I1, I2>>(op.input1.getType(), op.input2.getType()), op.windowSize,
+					length, op.timeStamp1, op.timeStamp2));
 		}
 
 		/**
@@ -97,13 +101,12 @@ public class StreamCrossOperator<I1, I2> extends
 			TypeInformation<R> outTypeInfo = TypeExtractor.getCrossReturnTypes(function,
 					op.input1.getType(), op.input2.getType());
 
-			CoStreamWindow<I1, I2, R> operator = new CoStreamWindow<I1, I2, R>(
-					new CrossWindowFunction<I1, I2, R>(clean(function)), op.windowSize,
-					op.slideInterval, op.timeStamp1, op.timeStamp2);
-
-			streamGraph.setOperator(id, operator);
+			CrossWindowFunction<I1, I2, R> crossWindowFunction = new CrossWindowFunction<I1, I2, R>(clean(function));
 
-			return ((SingleOutputStreamOperator<R, ?>) this).returns(outTypeInfo);
+			return new CrossWindow<I1, I2, R>(op, op.input1.connect(op.input2).addGeneralWindowCombine(
+					crossWindowFunction,
+					outTypeInfo, op.windowSize,
+					op.slideInterval, op.timeStamp1, op.timeStamp2));
 
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
index e18e14b..e48d707 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
@@ -31,7 +31,6 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.functions.co.JoinWindowFunction;
-import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
 import org.apache.flink.streaming.util.keys.KeySelectorUtil;
 
 public class StreamJoinOperator<I1, I2> extends
@@ -156,7 +155,7 @@ public class StreamJoinOperator<I1, I2> extends
 		 * @return A streaming join operator. Call {@link JoinedStream#with} to
 		 *         apply a custom wrapping
 		 */
-		public JoinedStream<I1, I2> equalTo(int... fields) {
+		public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(int... fields) {
 			keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields, type2),
 					type2, op.input1.getExecutionEnvironment().getConfig());
 			return createJoinOperator();
@@ -175,7 +174,7 @@ public class StreamJoinOperator<I1, I2> extends
 		 * @return A streaming join operator. Call {@link JoinedStream#with} to
 		 *         apply a custom wrapping
 		 */
-		public JoinedStream<I1, I2> equalTo(String... fields) {
+		public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(String... fields) {
 			this.keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields,
 					type2), type2, op.input1.getExecutionEnvironment().getConfig());
 			return createJoinOperator();
@@ -198,12 +197,12 @@ public class StreamJoinOperator<I1, I2> extends
 		 * @return A streaming join operator. Call {@link JoinedStream#with} to
 		 *         apply a custom wrapping
 		 */
-		public <K> JoinedStream<I1, I2> equalTo(KeySelector<I2, K> keySelector) {
+		public <K> JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(KeySelector<I2, K> keySelector) {
 			this.keys2 = keySelector;
 			return createJoinOperator();
 		}
 
-		private JoinedStream<I1, I2> createJoinOperator() {
+		private JoinedStream<I1, I2, Tuple2<I1, I2>> createJoinOperator() {
 
 			JoinFunction<I1, I2, Tuple2<I1, I2>> joinFunction = new DefaultJoinFunction<I1, I2>();
 
@@ -213,42 +212,44 @@ public class StreamJoinOperator<I1, I2> extends
 			TypeInformation<Tuple2<I1, I2>> outType = new TupleTypeInfo<Tuple2<I1, I2>>(
 					op.input1.getType(), op.input2.getType());
 
-			return new JoinedStream<I1, I2>(this, op.input1
+			return new JoinedStream<I1, I2, Tuple2<I1, I2>>(this, op.input1
 					.groupBy(keys1)
 					.connect(op.input2.groupBy(keys2))
 					.addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
 							op.slideInterval, op.timeStamp1, op.timeStamp2));
 		}
-	}
-
-	public static class JoinedStream<I1, I2> extends
-			SingleOutputStreamOperator<Tuple2<I1, I2>, JoinedStream<I1, I2>> {
-		private final JoinPredicate<I1, I2> predicate;
-
-		private JoinedStream(JoinPredicate<I1, I2> predicate, DataStream<Tuple2<I1, I2>> ds) {
-			super(ds);
-			this.predicate = predicate;
-		}
-
-		/**
-		 * Completes a stream join. </p> The resulting operator wraps each pair
-		 * of joining elements using the user defined {@link JoinFunction}
-		 * 
-		 * @return The joined data stream.
-		 */
-		@SuppressWarnings("unchecked")
-		public <OUT> SingleOutputStreamOperator<OUT, ?> with(JoinFunction<I1, I2, OUT> joinFunction) {
-
-			TypeInformation<OUT> outType = TypeExtractor.getJoinReturnTypes(joinFunction,
-					predicate.op.input1.getType(), predicate.op.input2.getType());
-
-			CoStreamWindow<I1, I2, OUT> operator = new CoStreamWindow<I1, I2, OUT>(
-					getJoinWindowFunction(joinFunction, predicate), predicate.op.windowSize,
-					predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2);
-
-			streamGraph.setOperator(id, operator);
 
-			return ((SingleOutputStreamOperator<OUT, ?>) this).returns(outType);
+		public static class JoinedStream<I1, I2, R> extends
+				SingleOutputStreamOperator<R, JoinedStream<I1, I2, R>> {
+			private final JoinPredicate<I1, I2> predicate;
+
+			private JoinedStream(JoinPredicate<I1, I2> predicate, DataStream<R> ds) {
+				super(ds.getExecutionEnvironment(), ds.getTransformation());
+				this.predicate = predicate;
+			}
+
+			/**
+			 * Completes a stream join. </p> The resulting operator wraps each pair
+			 * of joining elements using the user defined {@link JoinFunction}
+			 *
+			 * @return The joined data stream.
+			 */
+			@SuppressWarnings("unchecked")
+			public <OUT> SingleOutputStreamOperator<OUT, ?> with(JoinFunction<I1, I2, OUT> joinFunction) {
+
+				TypeInformation<OUT> outType = TypeExtractor.getJoinReturnTypes(joinFunction,
+						predicate.op.input1.getType(), predicate.op.input2.getType());
+
+				JoinWindowFunction<I1, I2, OUT> joinWindowFunction = getJoinWindowFunction(joinFunction, predicate);
+
+
+				return new JoinedStream<I1, I2, OUT>(
+						predicate, predicate.op.input1
+						.groupBy(predicate.keys1)
+						.connect(predicate.op.input2.groupBy(predicate.keys2))
+						.addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize,
+								predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2));
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
index 3fe7eb7..9da00f2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
@@ -40,8 +40,8 @@ public abstract class TemporalOperator<I1, I2, OP extends TemporalWindow<OP>> {
 		if (input1 == null || input2 == null) {
 			throw new NullPointerException();
 		}
-		this.input1 = input1.copy();
-		this.input2 = input2.copy();
+		this.input1 = input1;
+		this.input2 = input2;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 3efad93..58459b7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -43,9 +43,8 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	 */
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		JobExecutionResult result = ClusterUtil.runOnMiniCluster(this.streamGraph.getJobGraph(jobName), getParallelism(),
-				getConfig().isSysoutLoggingEnabled());
-		streamGraph.clear(); // clear graph to allow submitting another job via the same environment.
+		JobExecutionResult result = ClusterUtil.runOnMiniCluster(getStreamGraph().getJobGraph(), getParallelism(), getConfig().isSysoutLoggingEnabled());
+		transformations.clear();
 		return result;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 75e15d7..2f8938f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -84,15 +84,15 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public JobExecutionResult execute() throws ProgramInvocationException {
-
-		JobGraph jobGraph = streamGraph.getJobGraph();
+		JobGraph jobGraph = getStreamGraph().getJobGraph();
+		transformations.clear();
 		return executeRemotely(jobGraph);
 	}
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws ProgramInvocationException {
-
-		JobGraph jobGraph = streamGraph.getJobGraph(jobName);
+		JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
+		transformations.clear();
 		return executeRemotely(jobGraph);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
index 9bfeb2f..c2335d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
@@ -63,15 +63,16 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		currentEnvironment = null;
 
 		JobGraph jobGraph;
 		if (jobName == null) {
-			jobGraph = this.streamGraph.getJobGraph();
+			jobGraph = this.getStreamGraph().getJobGraph();
 		} else {
-			jobGraph = this.streamGraph.getJobGraph(jobName);
+			jobGraph = this.getStreamGraph().getJobGraph(jobName);
 		}
 
+		transformations.clear();
+
 		for (File file : jars) {
 			jobGraph.addJar(new Path(file.getAbsolutePath()));
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index fba4e28..a019a31 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.environment;
 
 import com.esotericsoftware.kryo.Serializer;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
@@ -60,8 +61,9 @@ import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
 import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
 import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.SplittableIterator;
 
@@ -86,22 +88,26 @@ public abstract class StreamExecutionEnvironment {
 
 	private ExecutionConfig config = new ExecutionConfig();
 
-	protected static StreamExecutionEnvironment currentEnvironment;
+	protected List<StreamTransformation<?>> transformations = Lists.newArrayList();
 
-	protected StreamGraph streamGraph;
+	protected boolean isChainingEnabled = true;
+
+	protected long checkpointInterval = -1; // disabled
+
+	protected CheckpointingMode checkpointingMode = null;
+
+	protected boolean forceCheckpointing = false;
+
+	protected StateHandleProvider<?> stateHandleProvider;
+
+	/** The environment of the context (local by default, cluster if invoked through command line) */
+	private static StreamExecutionEnvironmentFactory contextEnvironmentFactory;
 
 	// --------------------------------------------------------------------------------------------
 	// Constructor and Properties
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Constructor for creating StreamExecutionEnvironment
-	 */
-	protected StreamExecutionEnvironment() {
-		streamGraph = new StreamGraph(this);
-	}
-
-	/**
 	 * Gets the config object.
 	 */
 	public ExecutionConfig getConfig() {
@@ -221,10 +227,19 @@ public abstract class StreamExecutionEnvironment {
 	 * @return StreamExecutionEnvironment with chaining disabled.
 	 */
 	public StreamExecutionEnvironment disableOperatorChaining() {
-		streamGraph.setChaining(false);
+		this.isChainingEnabled = false;
 		return this;
 	}
 
+	/**
+	 * Returns whether operator chaining is enabled.
+	 *
+	 * @return {@code true} if chaining is enabled, false otherwise.
+	 */
+	public boolean isChainingEnabled() {
+		return isChainingEnabled;
+	}
+
 	// ------------------------------------------------------------------------
 	//  Checkpointing Settings
 	// ------------------------------------------------------------------------
@@ -275,10 +290,9 @@ public abstract class StreamExecutionEnvironment {
 		if (interval <= 0) {
 			throw new IllegalArgumentException("the checkpoint interval must be positive");
 		}
-		
-		streamGraph.setCheckpointingEnabled(true);
-		streamGraph.setCheckpointingInterval(interval);
-		streamGraph.setCheckpointingMode(mode);
+
+		this.checkpointInterval = interval;
+		this.checkpointingMode = mode;
 		return this;
 	}
 	
@@ -303,19 +317,9 @@ public abstract class StreamExecutionEnvironment {
 	 */
 	@Deprecated
 	public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) {
-		if (mode == null) {
-			throw new NullPointerException("checkpoint mode must not be null");
-		}
-		if (interval <= 0) {
-			throw new IllegalArgumentException("the checkpoint interval must be positive");
-		}
-		
-		streamGraph.setCheckpointingEnabled(true);
-		streamGraph.setCheckpointingInterval(interval);
-		streamGraph.setCheckpointingMode(mode);
-		if (force) {
-			streamGraph.forceCheckpoint();
-		}
+		this.enableCheckpointing(interval, mode);
+
+		this.forceCheckpointing = force;
 		return this;
 	}
 
@@ -334,12 +338,35 @@ public abstract class StreamExecutionEnvironment {
 	 * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.</p>
 	 */
 	public StreamExecutionEnvironment enableCheckpointing() {
-		streamGraph.setCheckpointingEnabled(true);
-		streamGraph.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
+		enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE);
 		return this;
 	}
 
 	/**
+	 * Returns the checkpointing interval or -1 if checkpointing is disabled.
+	 *
+	 * @return The checkpointing interval or -1
+	 */
+	public long getCheckpointInterval() {
+		return checkpointInterval;
+	}
+
+
+	/**
+	 * Returns whether checkpointing is force-enabled.
+	 */
+	public boolean isForceCheckpointing() {
+		return forceCheckpointing;
+	}
+
+	/**
+	 * Returns the {@link CheckpointingMode}.
+	 */
+	public CheckpointingMode getCheckpointingMode() {
+		return checkpointingMode;
+	}
+
+	/**
 	 * Sets the {@link StateHandleProvider} used for storing operator state
 	 * checkpoints when checkpointing is enabled.
 	 * <p>
@@ -348,11 +375,22 @@ public abstract class StreamExecutionEnvironment {
 	 * 
 	 */
 	public StreamExecutionEnvironment setStateHandleProvider(StateHandleProvider<?> provider) {
-		streamGraph.setStateHandleProvider(provider);
+		this.stateHandleProvider = provider;
 		return this;
 	}
 
 	/**
+	 * Returns the {@link org.apache.flink.runtime.state.StateHandle}
+	 *
+	 * @see #setStateHandleProvider(org.apache.flink.runtime.state.StateHandleProvider)
+	 *
+	 * @return The StateHandleProvider
+	 */
+	public StateHandleProvider<?> getStateHandleProvider() {
+		return stateHandleProvider;
+	}
+
+	/**
 	 * Sets the number of times that failed tasks are re-executed. A value of
 	 * zero effectively disables fault tolerance. A value of {@code -1}
 	 * indicates that the system default value (as defined in the configuration)
@@ -591,7 +629,7 @@ public abstract class StreamExecutionEnvironment {
 		
 		// must not have null elements and mixed elements
 		FromElementsFunction.checkCollection(data, typeInfo.getTypeClass());
-		
+
 		SourceFunction<OUT> function;
 		try {
 			function = new FromElementsFunction<OUT>(typeInfo.createSerializer(getConfig()), data);
@@ -599,7 +637,7 @@ public abstract class StreamExecutionEnvironment {
 		catch (IOException e) {
 			throw new RuntimeException(e.getMessage(), e);
 		}
-		return addSource(function, "Collection Source", typeInfo);
+		return addSource(function, "Collection Source", typeInfo).setParallelism(1);
 	}
 
 	/**
@@ -980,15 +1018,12 @@ public abstract class StreamExecutionEnvironment {
 	private <OUT> DataStreamSource<OUT> createInput(InputFormat<OUT, ?> inputFormat,
 			TypeInformation<OUT> typeInfo, String sourceName) {
 		FileSourceFunction<OUT> function = new FileSourceFunction<OUT>(inputFormat, typeInfo);
-		DataStreamSource<OUT> returnStream = addSource(function, sourceName).returns(typeInfo);
-		streamGraph.setInputFormat(returnStream.getId(), inputFormat);
-		return returnStream;
+		return addSource(function, sourceName).returns(typeInfo);
 	}
 
 	/**
-	 * Adds a data source with a custom type information thus opening a
-	 * {@link org.apache.flink.streaming.api.datastream.DataStream}. Only in very special cases does the user need
-	 * to support type information. Otherwise use {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)}
+	 * Adds a Data Source to the streaming topology.
+	 *
 	 * <p>
 	 * By default sources have a parallelism of 1. To enable parallel execution, the user defined source should
 	 * implement {@link org.apache.flink.streaming.api.functions.source.ParallelSourceFunction} or extend {@link
@@ -1078,10 +1113,9 @@ public abstract class StreamExecutionEnvironment {
 		boolean isParallel = function instanceof ParallelSourceFunction;
 
 		clean(function);
-		StreamOperator<OUT> sourceOperator = new StreamSource<OUT>(function);
+		StreamSource<OUT> sourceOperator = new StreamSource<OUT>(function);
 
-		return new DataStreamSource<OUT>(this, sourceName, typeInfo, sourceOperator,
-				isParallel, sourceName);
+		return new DataStreamSource<OUT>(this, typeInfo, sourceOperator, isParallel, sourceName);
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -1098,20 +1132,20 @@ public abstract class StreamExecutionEnvironment {
 	 * executed.
 	 */
 	public static StreamExecutionEnvironment getExecutionEnvironment() {
-		if (currentEnvironment != null) {
-			return currentEnvironment;
+		if (contextEnvironmentFactory != null) {
+			return contextEnvironmentFactory.createExecutionEnvironment();
 		}
+
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		if (env instanceof ContextEnvironment) {
 			ContextEnvironment ctx = (ContextEnvironment) env;
-			currentEnvironment = createContextEnvironment(ctx.getClient(), ctx.getJars(),
+			return createContextEnvironment(ctx.getClient(), ctx.getJars(),
 					ctx.getParallelism(), ctx.isWait());
 		} else if (env instanceof OptimizerPlanEnvironment | env instanceof PreviewPlanEnvironment) {
-			currentEnvironment = new StreamPlanEnvironment(env);
+			return new StreamPlanEnvironment(env);
 		} else {
 			return createLocalEnvironment();
 		}
-		return currentEnvironment;
 	}
 
 	private static StreamExecutionEnvironment createContextEnvironment(Client client,
@@ -1143,9 +1177,9 @@ public abstract class StreamExecutionEnvironment {
 	 * @return A local execution environment with the specified parallelism.
 	 */
 	public static LocalStreamEnvironment createLocalEnvironment(int parallelism) {
-		currentEnvironment = new LocalStreamEnvironment();
-		currentEnvironment.setParallelism(parallelism);
-		return (LocalStreamEnvironment) currentEnvironment;
+		LocalStreamEnvironment env = new LocalStreamEnvironment();
+		env.setParallelism(parallelism);
+		return env;
 	}
 
 	// TODO:fix cluster default parallelism
@@ -1172,8 +1206,8 @@ public abstract class StreamExecutionEnvironment {
 	 */
 	public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
 			String... jarFiles) {
-		currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles);
-		return currentEnvironment;
+		RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles);
+		return env;
 	}
 
 	/**
@@ -1199,9 +1233,9 @@ public abstract class StreamExecutionEnvironment {
 	 */
 	public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
 			int parallelism, String... jarFiles) {
-		currentEnvironment = new RemoteStreamEnvironment(host, port, jarFiles);
-		currentEnvironment.setParallelism(parallelism);
-		return currentEnvironment;
+		RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles);
+		env.setParallelism(parallelism);
+		return env;
 	}
 
 	/**
@@ -1239,7 +1273,11 @@ public abstract class StreamExecutionEnvironment {
 	 * @return The streamgraph representing the transformations
 	 */
 	public StreamGraph getStreamGraph() {
-		return streamGraph;
+		if (transformations.size() <= 0) {
+			throw new IllegalStateException("No operators defined in streaming topology. Cannot execute.");
+		}
+		StreamGraph result = StreamGraphGenerator.generate(this, transformations);
+		return result;
 	}
 
 	/**
@@ -1254,10 +1292,6 @@ public abstract class StreamExecutionEnvironment {
 		return getStreamGraph().getStreamingPlanAsJSON();
 	}
 
-	protected static void initializeFromFactory(StreamExecutionEnvironmentFactory eef) {
-		currentEnvironment = eef.createExecutionEnvironment();
-	}
-
 	/**
 	 * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
 	 * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
@@ -1270,4 +1304,28 @@ public abstract class StreamExecutionEnvironment {
 		return f;
 	}
 
+	/**
+	 * Adds an operator to the list of operators that should be executed when calling
+	 * {@link #execute}.
+	 *
+	 * <p>
+	 * When calling {@link #execute()} only the operators that where previously added to the list
+	 * are executed.
+	 *
+	 * <p>
+	 * This is not meant to be used by users. The API methods that create operators must call
+	 * this method.
+	 */
+	public void addOperator(StreamTransformation<?> transformation) {
+		Preconditions.checkNotNull(transformation, "Sinks must not be null.");
+		this.transformations.add(transformation);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Methods to control the context and local environments for execution from packaged programs
+	// --------------------------------------------------------------------------------------------
+
+	protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) {
+		contextEnvironmentFactory = ctx;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
index 02fccd0..8c1408e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
@@ -24,6 +24,7 @@ import org.apache.flink.client.program.Client.OptimizerPlanEnvironment;
 import org.apache.flink.client.program.PackagedProgram.PreviewPlanEnvironment;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.streaming.api.graph.StreamGraph;
 
 public class StreamPlanEnvironment extends StreamExecutionEnvironment {
 
@@ -55,10 +56,12 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		currentEnvironment = null;
 
+		StreamGraph streamGraph = getStreamGraph();
 		streamGraph.setJobName(jobName);
 
+		transformations.clear();
+
 		if (env instanceof OptimizerPlanEnvironment) {
 			((OptimizerPlanEnvironment) env).setPlan(streamGraph);
 		} else if (env instanceof PreviewPlanEnvironment) {

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
index cf08e5a..253c076 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
@@ -134,4 +134,12 @@ public class FileSourceFunction<OUT> extends RichParallelSourceFunction<OUT> {
 		isRunning = false;
 	}
 
+
+	/**
+	 * Returns the {@code InputFormat}. This is only needed because we need to set the input
+	 * split assigner on the {@code StreamGraph}.
+	 */
+	public InputFormat<OUT, InputSplit> getFormat() {
+		return format;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
index bc20fff..743ee4a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.graph;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -51,7 +52,19 @@ public class JSONGenerator {
 		JSONArray nodes = new JSONArray();
 		json.put("nodes", nodes);
 		List<Integer> operatorIDs = new ArrayList<Integer>(streamGraph.getVertexIDs());
-		Collections.sort(operatorIDs);
+		Collections.sort(operatorIDs, new Comparator<Integer>() {
+			@Override
+			public int compare(Integer o1, Integer o2) {
+				// put sinks at the back
+				if (streamGraph.getSinkIDs().contains(o1)) {
+					return 1;
+				} else if (streamGraph.getSinkIDs().contains(o2)) {
+					return -1;
+				} else {
+					return o1 - o2;
+				}
+			}
+		});
 		visit(nodes, operatorIDs, new HashMap<Integer, Integer>());
 		return json.toString();
 	}
@@ -87,7 +100,7 @@ public class JSONGenerator {
 			for (StreamEdge inEdge : vertex.getInEdges()) {
 				int operator = inEdge.getSourceId();
 
-				if (streamGraph.vertexIDtoLoop.containsKey(operator)) {
+				if (streamGraph.vertexIDtoLoopTimeout.containsKey(operator)) {
 					iterationHead = operator;
 				}
 			}
@@ -119,7 +132,7 @@ public class JSONGenerator {
 		toVisit.remove(vertexID);
 
 		// Ignoring head and tail to avoid redundancy
-		if (!streamGraph.vertexIDtoLoop.containsKey(vertexID)) {
+		if (!streamGraph.vertexIDtoLoopTimeout.containsKey(vertexID)) {
 			JSONObject obj = new JSONObject();
 			jsonArray.put(obj);
 			decorateNode(vertexID, obj);
@@ -131,7 +144,7 @@ public class JSONGenerator {
 
 				if (edgeRemapings.keySet().contains(inputID)) {
 					decorateEdge(inEdges, vertexID, inputID, inputID);
-				} else if (!streamGraph.vertexIDtoLoop.containsKey(inputID)) {
+				} else if (!streamGraph.vertexIDtoLoopTimeout.containsKey(inputID)) {
 					decorateEdge(iterationInEdges, vertexID, inputID, inputID);
 				}
 			}
@@ -147,8 +160,7 @@ public class JSONGenerator {
 		JSONObject input = new JSONObject();
 		inputArray.put(input);
 		input.put(ID, mappedInputID);
-		input.put(SHIP_STRATEGY, streamGraph.getStreamEdge(inputID, vertexID).getPartitioner()
-				.getStrategy());
+		input.put(SHIP_STRATEGY, streamGraph.getStreamEdge(inputID, vertexID).getPartitioner());
 		input.put(SIDE, (inputArray.length() == 0) ? "first" : "second");
 	}
 
@@ -161,8 +173,10 @@ public class JSONGenerator {
 
 		if (streamGraph.getSourceIDs().contains(vertexID)) {
 			node.put(PACT, "Data Source");
+		} else if (streamGraph.getSinkIDs().contains(vertexID)) {
+			node.put(PACT, "Data Sink");
 		} else {
-			node.put(PACT, "Data Stream");
+			node.put(PACT, "Operator");
 		}
 
 		StreamOperator<?> operator = streamGraph.getStreamNode(vertexID).getOperator();

http://git-wip-us.apache.org/repos/asf/flink/blob/bac21bf5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
index 4f19db6..2c422d9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
@@ -257,29 +257,6 @@ public class StreamConfig implements Serializable {
 		return config.getLong(ITERATON_WAIT, 0);
 	}
 
-	public void setSelectedNames(Integer output, List<String> selected) {
-		if (selected == null) {
-			selected = new ArrayList<String>();
-		}
-
-		try {
-			InstantiationUtil.writeObjectToConfig(selected, this.config, OUTPUT_NAME + output);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize OutputSelector for name \"" + output+ "\".", e);
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	public List<String> getSelectedNames(Integer output, ClassLoader cl) {
-		List<String> selectedNames;
-		try {
-			selectedNames = (List<String>) InstantiationUtil.readObjectFromConfig(this.config, OUTPUT_NAME + output, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Cannot deserialize OutputSelector for name \"" + output + "\".", e);
-		}
-		return selectedNames == null ? new ArrayList<String>() : selectedNames;
-	}
-
 	public void setNumberOfInputs(int numberOfInputs) {
 		config.setInteger(NUMBER_OF_INPUTS, numberOfInputs);
 	}