You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mb...@apache.org on 2014/09/24 21:51:34 UTC

[01/12] git commit: [FLINK-1121] [streaming] minBy and maxBy operators added to streaming api

Repository: incubator-flink
Updated Branches:
  refs/heads/master a3b02840d -> cb81319d9


[FLINK-1121] [streaming] minBy and maxBy operators added to streaming api


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

Branch: refs/heads/master
Commit: 70464bb0a44f682c155fdfdd2a6b0a6cc1203663
Parents: 30ac9fe
Author: Gyula Fora <gy...@apache.org>
Authored: Wed Sep 24 16:57:36 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md                         |   6 +-
 .../api/datastream/BatchedDataStream.java       |  69 +++++++-
 .../streaming/api/datastream/DataStream.java    |  66 +++++++
 .../api/datastream/GroupedDataStream.java       |  71 ++++++++
 .../aggregation/MaxByAggregationFunction.java   |  37 ++++
 .../aggregation/MinByAggregationFunction.java   |  55 ++++++
 .../streaming/api/AggregationFunctionTest.java  | 177 +++++++++++++++++--
 7 files changed, 460 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index 27a32ba..37ff90d 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -246,9 +246,11 @@ When the reduce operator is applied on a grouped data stream, the user-defined `
 
 The Flink Streaming API supports different types of aggregation operators similarly to the core API. For grouped data streams the aggregations work in a grouped fashion.
 
-Types of aggregations: `sum(fieldPosition)`, `min(fieldPosition)`, `max(fieldPosition)`
+Types of aggregations: `sum(fieldPosition)`, `min(fieldPosition)`, `max(fieldPosition)`, `minBy(fieldPosition, first)`, `maxBy(fieldPosition, first)`
 
-For every incoming tuple the selected field is replaced with the current aggregated value. If the aggregations are used without defining field position, position `0` is used as default. 
+With `sum`, `min`, and `max` for every incoming tuple the selected field is replaced with the current aggregated value. If the aggregations are used without defining field position, position `0` is used as default. 
+
+With `minBy` and `maxBy` the output of the operator is the element with the current minimal or maximal value at the given fieldposition. If more components share the minimum or maximum value, the user can decide if the operator should return the first or last element. This can be set by the `first` boolean parameter.
 
 ### Window/Batch operators
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
index 51f1467..e8e3f31 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
@@ -24,7 +24,9 @@ import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MaxByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MinByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
 import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
@@ -118,8 +120,8 @@ public class BatchedDataStream<OUT> {
 	 */
 	public <R> SingleOutputStreamOperator<R, ?> reduceGroup(GroupReduceFunction<OUT, R> reducer) {
 		return dataStream.addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
-				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer, GroupReduceFunction.class,
-				1), getGroupReduceInvokable(reducer));
+				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
+				GroupReduceFunction.class, 1), getGroupReduceInvokable(reducer));
 	}
 
 	/**
@@ -160,6 +162,38 @@ public class BatchedDataStream<OUT> {
 	}
 
 	/**
+	 * Applies an aggregation that gives the minimum element of every sliding
+	 * batch/window of the data stream by the given position. If more elements
+	 * have the same minimum value the operator returns the first element by
+	 * default.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every sliding
+	 * batch/window of the data stream by the given position. If more elements
+	 * have the same minimum value the operator returns either the first or last
+	 * one depending on the parameter setting.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            minimum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy, boolean first) {
+		dataStream.checkFieldRange(positionToMinBy);
+		return aggregate(new MinByAggregationFunction<OUT>(positionToMinBy, first));
+	}
+
+	/**
 	 * Syntactic sugar for min(0)
 	 * 
 	 * @return The transformed DataStream.
@@ -182,6 +216,37 @@ public class BatchedDataStream<OUT> {
 	}
 
 	/**
+	 * Applies an aggregation that gives the maximum element of every sliding
+	 * batch/window of the data stream by the given position. If more elements
+	 * have the same maximum value the operator returns the first by default.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every sliding
+	 * batch/window of the data stream by the given position. If more elements
+	 * have the same maximum value the operator returns either the first or last
+	 * one depending on the parameter setting.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy, boolean first) {
+		dataStream.checkFieldRange(positionToMaxBy);
+		return aggregate(new MaxByAggregationFunction<OUT>(positionToMaxBy, first));
+	}
+
+	/**
 	 * Syntactic sugar for max(0)
 	 * 
 	 * @return The transformed DataStream.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 423de4b..8ff8c54 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -39,7 +39,9 @@ import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MaxByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MinByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
 import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
 import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -553,6 +555,38 @@ public class DataStream<OUT> {
 	}
 
 	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position, if more elements have the minimum
+	 * value at the given position, the operator returns the first one by
+	 * default.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position, if more elements have the minimum
+	 * value at the given position, the operator returns either the first or
+	 * last one, depending on the parameter set.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            minimal value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy, boolean first) {
+		checkFieldRange(positionToMinBy);
+		return aggregate(new MinByAggregationFunction<OUT>(positionToMinBy, first));
+	}
+
+	/**
 	 * Syntactic sugar for min(0)
 	 * 
 	 * @return The transformed DataStream.
@@ -575,6 +609,38 @@ public class DataStream<OUT> {
 	}
 
 	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position, if more elements have the maximum
+	 * value at the given position, the operator returns the first one by
+	 * default.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position, if more elements have the maximum
+	 * value at the given position, the operator returns either the first or
+	 * last one, depending on the parameter set.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize.
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy, boolean first) {
+		checkFieldRange(positionToMaxBy);
+		return aggregate(new MaxByAggregationFunction<OUT>(positionToMaxBy, first));
+	}
+
+	/**
 	 * Syntactic sugar for max(0)
 	 * 
 	 * @return The transformed DataStream.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
index 3a61a35..af2f186 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
@@ -75,6 +75,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	 *            The position in the data point to sum
 	 * @return The transformed DataStream.
 	 */
+	@Override
 	public SingleOutputStreamOperator<OUT, ?> sum(final int positionToSum) {
 		return super.sum(positionToSum);
 	}
@@ -88,11 +89,46 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	 *            The position in the data point to minimize
 	 * @return The transformed DataStream.
 	 */
+	@Override
 	public SingleOutputStreamOperator<OUT, ?> min(final int positionToMin) {
 		return super.min(positionToMin);
 	}
 
 	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position for each group on a grouped data
+	 * stream. If more elements have the minimum value at the given position,
+	 * the operator returns the first one by default.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	@Override
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy) {
+		return super.minBy(positionToMinBy);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position for each group on a grouped data
+	 * stream. If more elements have the minimum value at the given position,
+	 * the operator returns either the first or last one depending on the
+	 * parameters.
+	 * 
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	@Override
+	public SingleOutputStreamOperator<OUT, ?> minBy(int positionToMinBy, boolean first) {
+		return super.minBy(positionToMinBy, first);
+	}
+
+	/**
 	 * Applies an aggregation that gives the maximum of the grouped data stream
 	 * at the given position, grouped by the given key position. Input values
 	 * with the same key will be maximized.
@@ -101,10 +137,45 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	 *            The position in the data point to maximize
 	 * @return The transformed DataStream.
 	 */
+	@Override
 	public SingleOutputStreamOperator<OUT, ?> max(final int positionToMax) {
 		return super.max(positionToMax);
 	}
 
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position for each group on a grouped data
+	 * stream. If more elements have the maximum value at the given position,
+	 * the operator returns the first one by default.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	@Override
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy) {
+		return super.maxBy(positionToMaxBy);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position for each group on a grouped data
+	 * stream. If more elements have the maximum value at the given position,
+	 * the operator returns either the first or last one depending on the
+	 * parameters.
+	 * 
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	@Override
+	public SingleOutputStreamOperator<OUT, ?> maxBy(int positionToMaxBy, boolean first) {
+		return super.maxBy(positionToMaxBy, first);
+	}
+
 	@Override
 	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MaxByAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MaxByAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MaxByAggregationFunction.java
new file mode 100644
index 0000000..274c8b6
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MaxByAggregationFunction.java
@@ -0,0 +1,37 @@
+/*
+ * 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.function.aggregation;
+
+public class MaxByAggregationFunction<T> extends MinByAggregationFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	public MaxByAggregationFunction(int pos, boolean first) {
+		super(pos, first);
+	}
+
+	@Override
+	public <R> boolean isExtremal(Comparable<R> o1, R o2) {
+		if (first) {
+			return o1.compareTo(o2) >= 0;
+		} else {
+			return o1.compareTo(o2) > 0;
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MinByAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MinByAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MinByAggregationFunction.java
new file mode 100644
index 0000000..a4a328c
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/MinByAggregationFunction.java
@@ -0,0 +1,55 @@
+/*
+ * 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.function.aggregation;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+public class MinByAggregationFunction<T> extends ComparableAggregationFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+	protected boolean first;
+
+	public MinByAggregationFunction(int pos, boolean first) {
+		super(pos);
+		this.first = first;
+	}
+
+	@Override
+	public <R> void compare(Tuple tuple1, Tuple tuple2) throws InstantiationException,
+			IllegalAccessException {
+
+		Comparable<R> o1 = tuple1.getField(position);
+		R o2 = tuple2.getField(position);
+
+		if (isExtremal(o1, o2)) {
+			returnTuple = tuple1;
+		} else {
+			returnTuple = tuple2;
+		}
+	}
+
+	@Override
+	public <R> boolean isExtremal(Comparable<R> o1, R o2) {
+		if (first) {
+			return o1.compareTo(o2) <= 0;
+		} else {
+			return o1.compareTo(o2) < 0;
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/70464bb0/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
index d48f8ad..1f86ce1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
@@ -26,7 +26,9 @@ import java.util.List;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.function.aggregation.MaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MaxByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.MinByAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.GroupedReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
@@ -49,7 +51,7 @@ public class AggregationFunctionTest {
 		List<Tuple2<Integer, Integer>> expectedGroupMaxList = new ArrayList<Tuple2<Integer, Integer>>();
 
 		List<Integer> simpleInput = new ArrayList<Integer>();
-		
+
 		int groupedSum0 = 0;
 		int groupedSum1 = 0;
 		int groupedSum2 = 0;
@@ -86,16 +88,14 @@ public class AggregationFunctionTest {
 		SumAggregationFunction<Tuple2<Integer, Integer>> sumFunction = SumAggregationFunction
 				.getSumFunction(1, Integer.class);
 		@SuppressWarnings("unchecked")
-		SumAggregationFunction<Integer> sumFunction0 = SumAggregationFunction
-				.getSumFunction(0, Integer.class);
+		SumAggregationFunction<Integer> sumFunction0 = SumAggregationFunction.getSumFunction(0,
+				Integer.class);
 		MinAggregationFunction<Tuple2<Integer, Integer>> minFunction = new MinAggregationFunction<Tuple2<Integer, Integer>>(
 				1);
-		MinAggregationFunction<Integer> minFunction0 = new MinAggregationFunction<Integer>(
-				0);
+		MinAggregationFunction<Integer> minFunction0 = new MinAggregationFunction<Integer>(0);
 		MaxAggregationFunction<Tuple2<Integer, Integer>> maxFunction = new MaxAggregationFunction<Tuple2<Integer, Integer>>(
 				1);
-		MaxAggregationFunction<Integer> maxFunction0 = new MaxAggregationFunction<Integer>(
-				0);
+		MaxAggregationFunction<Integer> maxFunction0 = new MaxAggregationFunction<Integer>(0);
 
 		List<Tuple2<Integer, Integer>> sumList = MockInvokable.createAndExecute(
 				new StreamReduceInvokable<Tuple2<Integer, Integer>>(sumFunction), getInputList());
@@ -107,13 +107,16 @@ public class AggregationFunctionTest {
 				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxFunction), getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedSumList = MockInvokable.createAndExecute(
-				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0),
+				getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedMinList = MockInvokable.createAndExecute(
-				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0),
+				getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedMaxList = MockInvokable.createAndExecute(
-				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0),
+				getInputList());
 
 		assertEquals(expectedSumList, sumList);
 		assertEquals(expectedMinList, minList);
@@ -121,31 +124,171 @@ public class AggregationFunctionTest {
 		assertEquals(expectedGroupSumList, groupedSumList);
 		assertEquals(expectedGroupMinList, groupedMinList);
 		assertEquals(expectedGroupMaxList, groupedMaxList);
-		assertEquals(expectedSumList0, MockInvokable.createAndExecute(new StreamReduceInvokable<Integer>(sumFunction0),simpleInput ));
-		assertEquals(expectedMinList0, MockInvokable.createAndExecute(new StreamReduceInvokable<Integer>(minFunction0),simpleInput ));
-		assertEquals(expectedMaxList0, MockInvokable.createAndExecute(new StreamReduceInvokable<Integer>(maxFunction0),simpleInput ));
-
+		assertEquals(expectedSumList0, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Integer>(sumFunction0), simpleInput));
+		assertEquals(expectedMinList0, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Integer>(minFunction0), simpleInput));
+		assertEquals(expectedMaxList0, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Integer>(maxFunction0), simpleInput));
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
 		try {
 			env.generateSequence(1, 100).min(1);
 			fail();
 		} catch (Exception e) {
-			//Nothing to do here
+			// Nothing to do here
 		}
 		try {
 			env.generateSequence(1, 100).min(2);
 			fail();
 		} catch (Exception e) {
-			//Nothing to do here
+			// Nothing to do here
 		}
 		try {
 			env.generateSequence(1, 100).min(3);
 			fail();
 		} catch (Exception e) {
-			//Nothing to do here
+			// Nothing to do here
 		}
 
+		MaxByAggregationFunction<Tuple2<Integer, Integer>> maxByFunctionFirst = new MaxByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, true);
+		MaxByAggregationFunction<Tuple2<Integer, Integer>> maxByFunctionLast = new MaxByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, false);
+
+		MinByAggregationFunction<Tuple2<Integer, Integer>> minByFunctionFirst = new MinByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, true);
+		MinByAggregationFunction<Tuple2<Integer, Integer>> minByFunctionLast = new MinByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, false);
+
+		List<Tuple2<Integer, Integer>> maxByFirstExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(1, 1));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+
+		List<Tuple2<Integer, Integer>> maxByLastExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(1, 1));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 8));
+
+		List<Tuple2<Integer, Integer>> minByFirstExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+
+		List<Tuple2<Integer, Integer>> minByLastExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+
+		assertEquals(maxByFirstExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxByFunctionFirst),
+				getInputList()));
+		assertEquals(maxByLastExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxByFunctionLast),
+				getInputList()));
+		assertEquals(minByLastExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minByFunctionLast),
+				getInputList()));
+		assertEquals(minByFirstExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minByFunctionFirst),
+				getInputList()));
+
+	}
+
+	@Test
+	public void minMaxByTest() {
+
+		MaxByAggregationFunction<Tuple2<Integer, Integer>> maxByFunctionFirst = new MaxByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, true);
+		MaxByAggregationFunction<Tuple2<Integer, Integer>> maxByFunctionLast = new MaxByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, false);
+
+		MinByAggregationFunction<Tuple2<Integer, Integer>> minByFunctionFirst = new MinByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, true);
+		MinByAggregationFunction<Tuple2<Integer, Integer>> minByFunctionLast = new MinByAggregationFunction<Tuple2<Integer, Integer>>(
+				0, false);
+
+		List<Tuple2<Integer, Integer>> maxByFirstExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(1, 1));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByFirstExpected.add(new Tuple2<Integer, Integer>(2, 2));
+
+		List<Tuple2<Integer, Integer>> maxByLastExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(1, 1));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 2));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 5));
+		maxByLastExpected.add(new Tuple2<Integer, Integer>(2, 8));
+
+		List<Tuple2<Integer, Integer>> minByFirstExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByFirstExpected.add(new Tuple2<Integer, Integer>(0, 0));
+
+		List<Tuple2<Integer, Integer>> minByLastExpected = new ArrayList<Tuple2<Integer, Integer>>();
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 0));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 3));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+		minByLastExpected.add(new Tuple2<Integer, Integer>(0, 6));
+
+		assertEquals(maxByFirstExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxByFunctionFirst),
+				getInputList()));
+		assertEquals(maxByLastExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxByFunctionLast),
+				getInputList()));
+		assertEquals(minByLastExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minByFunctionLast),
+				getInputList()));
+		assertEquals(minByFirstExpected, MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minByFunctionFirst),
+				getInputList()));
 	}
 
 	private List<Tuple2<Integer, Integer>> getInputList() {


[04/12] git commit: [FLINK-1102] [streaming] Projection operator added to DataStream

Posted by mb...@apache.org.
[FLINK-1102] [streaming] Projection operator added to DataStream

Conflicts:
	flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java

Conflicts:
	flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
	flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
	flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
	flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java


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

Branch: refs/heads/master
Commit: 4175dca89ea89522ff474b0c6c861516d03ee064
Parents: ad98337
Author: Gyula Fora <gy...@apache.org>
Authored: Tue Sep 23 14:50:35 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    |   32 +-
 .../flink/streaming/api/StreamConfig.java       |   14 +-
 .../api/datastream/ConnectedDataStream.java     |    6 +-
 .../streaming/api/datastream/DataStream.java    |   46 +-
 .../api/datastream/DataStreamSink.java          |    4 +-
 .../api/datastream/DataStreamSource.java        |    4 +-
 .../datastream/SingleOutputStreamOperator.java  |    4 +-
 .../api/datastream/StreamProjection.java        | 1469 ++++++++++++++++++
 .../environment/StreamExecutionEnvironment.java |    8 +-
 .../api/invokable/StreamInvokable.java          |    8 +-
 .../operator/BatchReduceInvokable.java          |    2 +-
 .../invokable/operator/ProjectInvokable.java    |   65 +
 .../util/serialization/FunctionTypeWrapper.java |    2 +-
 .../util/serialization/ObjectTypeWrapper.java   |    2 +-
 .../util/serialization/ProjectTypeWrapper.java  |   70 +
 .../serialization/TypeSerializerWrapper.java    |   38 -
 .../util/serialization/TypeWrapper.java         |   38 +
 .../api/invokable/operator/ProjectTest.java     |   66 +
 .../api/streamvertex/StreamVertexTest.java      |    5 +
 .../flink/streaming/util/MockCollector.java     |    8 +-
 .../serialization/TypeSerializationTest.java    |    8 +-
 21 files changed, 1801 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index a04dbaa..3377ee0 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -39,7 +39,7 @@ import org.apache.flink.streaming.api.streamvertex.StreamIterationTail;
 import org.apache.flink.streaming.api.streamvertex.StreamVertex;
 import org.apache.flink.streaming.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,10 +64,10 @@ public class JobGraphBuilder {
 	private Map<String, List<StreamPartitioner<?>>> connectionTypes;
 	private Map<String, String> operatorNames;
 	private Map<String, StreamInvokable<?, ?>> invokableObjects;
-	private Map<String, TypeSerializerWrapper<?>> typeWrapperIn1;
-	private Map<String, TypeSerializerWrapper<?>> typeWrapperIn2;
-	private Map<String, TypeSerializerWrapper<?>> typeWrapperOut1;
-	private Map<String, TypeSerializerWrapper<?>> typeWrapperOut2;
+	private Map<String, TypeWrapper<?>> typeWrapperIn1;
+	private Map<String, TypeWrapper<?>> typeWrapperIn2;
+	private Map<String, TypeWrapper<?>> typeWrapperOut1;
+	private Map<String, TypeWrapper<?>> typeWrapperOut2;
 	private Map<String, byte[]> serializedFunctions;
 	private Map<String, byte[]> outputSelectors;
 	private Map<String, Class<? extends AbstractInvokable>> vertexClasses;
@@ -103,10 +103,10 @@ public class JobGraphBuilder {
 		connectionTypes = new HashMap<String, List<StreamPartitioner<?>>>();
 		operatorNames = new HashMap<String, String>();
 		invokableObjects = new HashMap<String, StreamInvokable<?, ?>>();
-		typeWrapperIn1 = new HashMap<String, TypeSerializerWrapper<?>>();
-		typeWrapperIn2 = new HashMap<String, TypeSerializerWrapper<?>>();
-		typeWrapperOut1 = new HashMap<String, TypeSerializerWrapper<?>>();
-		typeWrapperOut2 = new HashMap<String, TypeSerializerWrapper<?>>();
+		typeWrapperIn1 = new HashMap<String, TypeWrapper<?>>();
+		typeWrapperIn2 = new HashMap<String, TypeWrapper<?>>();
+		typeWrapperOut1 = new HashMap<String, TypeWrapper<?>>();
+		typeWrapperOut2 = new HashMap<String, TypeWrapper<?>>();
 		serializedFunctions = new HashMap<String, byte[]>();
 		outputSelectors = new HashMap<String, byte[]>();
 		vertexClasses = new HashMap<String, Class<? extends AbstractInvokable>>();
@@ -156,8 +156,8 @@ public class JobGraphBuilder {
 	 *            Number of parallel instances created
 	 */
 	public <IN, OUT> void addStreamVertex(String vertexName,
-			StreamInvokable<IN, OUT> invokableObject, TypeSerializerWrapper<?> inTypeWrapper,
-			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
+			StreamInvokable<IN, OUT> invokableObject, TypeWrapper<?> inTypeWrapper,
+			TypeWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
 		addVertex(vertexName, StreamVertex.class, invokableObject, operatorName,
@@ -241,8 +241,8 @@ public class JobGraphBuilder {
 
 	public <IN1, IN2, OUT> void addCoTask(String vertexName,
 			CoInvokable<IN1, IN2, OUT> taskInvokableObject,
-			TypeSerializerWrapper<?> in1TypeWrapper, TypeSerializerWrapper<?> in2TypeWrapper,
-			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
+			TypeWrapper<?> in1TypeWrapper, TypeWrapper<?> in2TypeWrapper,
+			TypeWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
 		addVertex(vertexName, CoStreamVertex.class, taskInvokableObject, operatorName,
@@ -290,9 +290,9 @@ public class JobGraphBuilder {
 		iterationTailCount.put(vertexName, 0);
 	}
 
-	private void addTypeWrappers(String vertexName, TypeSerializerWrapper<?> in1,
-			TypeSerializerWrapper<?> in2, TypeSerializerWrapper<?> out1,
-			TypeSerializerWrapper<?> out2) {
+	private void addTypeWrappers(String vertexName, TypeWrapper<?> in1,
+			TypeWrapper<?> in2, TypeWrapper<?> out1,
+			TypeWrapper<?> out2) {
 		typeWrapperIn1.put(vertexName, in1);
 		typeWrapperIn2.put(vertexName, in2);
 		typeWrapperOut1.put(vertexName, out1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 6fac391..42c1adf 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -31,7 +31,7 @@ import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.streamvertex.StreamVertexException;
 import org.apache.flink.streaming.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 public class StreamConfig {
 	private static final String INPUT_TYPE = "inputType_";
@@ -79,19 +79,19 @@ public class StreamConfig {
 	private static final String TYPE_WRAPPER_OUT_1 = "typeWrapper_out_1";
 	private static final String TYPE_WRAPPER_OUT_2 = "typeWrapper_out_2";
 
-	public void setTypeWrapperIn1(TypeSerializerWrapper<?> typeWrapper) {
+	public void setTypeWrapperIn1(TypeWrapper<?> typeWrapper) {
 		setTypeWrapper(TYPE_WRAPPER_IN_1, typeWrapper);
 	}
 
-	public void setTypeWrapperIn2(TypeSerializerWrapper<?> typeWrapper) {
+	public void setTypeWrapperIn2(TypeWrapper<?> typeWrapper) {
 		setTypeWrapper(TYPE_WRAPPER_IN_2, typeWrapper);
 	}
 
-	public void setTypeWrapperOut1(TypeSerializerWrapper<?> typeWrapper) {
+	public void setTypeWrapperOut1(TypeWrapper<?> typeWrapper) {
 		setTypeWrapper(TYPE_WRAPPER_OUT_1, typeWrapper);
 	}
 
-	public void setTypeWrapperOut2(TypeSerializerWrapper<?> typeWrapper) {
+	public void setTypeWrapperOut2(TypeWrapper<?> typeWrapper) {
 		setTypeWrapper(TYPE_WRAPPER_OUT_2, typeWrapper);
 	}
 
@@ -111,7 +111,7 @@ public class StreamConfig {
 		return getTypeInfo(TYPE_WRAPPER_OUT_2);
 	}
 
-	private void setTypeWrapper(String key, TypeSerializerWrapper<?> typeWrapper) {
+	private void setTypeWrapper(String key, TypeWrapper<?> typeWrapper) {
 		config.setBytes(key, SerializationUtils.serialize(typeWrapper));
 	}
 
@@ -123,7 +123,7 @@ public class StreamConfig {
 			throw new RuntimeException("TypeSerializationWrapper must be set");
 		}
 
-		TypeSerializerWrapper<T> typeWrapper = (TypeSerializerWrapper<T>) SerializationUtils
+		TypeWrapper<T> typeWrapper = (TypeWrapper<T>) SerializationUtils
 				.deserialize(serializedWrapper);
 		if (typeWrapper != null) {
 			return typeWrapper.getTypeInfo();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index 256f470..d491fad 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -41,7 +41,7 @@ import org.apache.flink.streaming.api.invokable.operator.co.CoWindowGroupReduceI
 import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
 import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * The ConnectedDataStream represents a stream for two different data types. It
@@ -431,8 +431,8 @@ public class ConnectedDataStream<IN1, IN2> {
 	}
 
 	protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,
-			final Function function, TypeSerializerWrapper<IN1> in1TypeWrapper,
-			TypeSerializerWrapper<IN2> in2TypeWrapper, TypeSerializerWrapper<OUT> outTypeWrapper,
+			final Function function, TypeWrapper<IN1> in1TypeWrapper,
+			TypeWrapper<IN2> in2TypeWrapper, TypeWrapper<OUT> outTypeWrapper,
 			CoInvokable<IN1, IN2, OUT> functionInvokable) {
 
 		@SuppressWarnings({ "unchecked", "rawtypes" })

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 23bc80d..423de4b 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -33,6 +33,7 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -63,7 +64,7 @@ import org.apache.flink.streaming.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * A DataStream represents a stream of elements of the same type. A DataStream
@@ -88,7 +89,7 @@ public class DataStream<OUT> {
 	protected List<String> userDefinedNames;
 	protected boolean selectAll;
 	protected StreamPartitioner<OUT> partitioner;
-	protected final TypeSerializerWrapper<OUT> outTypeWrapper;
+	protected final TypeWrapper<OUT> outTypeWrapper;
 	protected List<DataStream<OUT>> mergedStreams;
 
 	protected final JobGraphBuilder jobGraphBuilder;
@@ -105,7 +106,7 @@ public class DataStream<OUT> {
 	 *            Type of the output
 	 */
 	public DataStream(StreamExecutionEnvironment environment, String operatorType,
-			TypeSerializerWrapper<OUT> outTypeWrapper) {
+			TypeWrapper<OUT> outTypeWrapper) {
 		if (environment == null) {
 			throw new NullPointerException("context is null");
 		}
@@ -384,6 +385,29 @@ public class DataStream<OUT> {
 	}
 
 	/**
+	 * 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
+	 * fields.</br> This method returns a {@link StreamProjection} on which
+	 * {@link StreamProjection#types()} needs to be called to completed the
+	 * transformation.
+	 * 
+	 * @param fieldIndexes
+	 *            The field indexes of the input tuples that are retained. The
+	 *            order of fields in the output tuple corresponds to the order
+	 *            of field indexes.
+	 * @return A StreamProjection that needs to be converted into a DataStream
+	 *         to complete the project transformation by calling
+	 *         {@link StreamProjection#types()}.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public StreamProjection<OUT> project(int... fieldIndexes) {
+		return new StreamProjection<OUT>(this.copy(), fieldIndexes);
+	}
+
+	/**
 	 * Groups the elements of a {@link DataStream} by the given key position to
 	 * be used with grouped operators like
 	 * {@link GroupedDataStream#reduce(ReduceFunction)}
@@ -565,8 +589,8 @@ public class DataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<Long, ?> count() {
-		TypeSerializerWrapper<OUT> inTypeWrapper = outTypeWrapper;
-		TypeSerializerWrapper<Long> outTypeWrapper = new ObjectTypeWrapper<Long>(new Long(0));
+		TypeWrapper<OUT> inTypeWrapper = outTypeWrapper;
+		TypeWrapper<Long> outTypeWrapper = new ObjectTypeWrapper<Long>(new Long(0));
 
 		return addFunction("counter", null, inTypeWrapper, outTypeWrapper,
 				new CounterInvokable<OUT>());
@@ -968,8 +992,8 @@ public class DataStream<OUT> {
 	 * @return the data stream constructed
 	 */
 	protected <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
-			final Function function, TypeSerializerWrapper<OUT> inTypeWrapper,
-			TypeSerializerWrapper<R> outTypeWrapper, StreamInvokable<OUT, R> functionInvokable) {
+			final Function function, TypeWrapper<OUT> inTypeWrapper, TypeWrapper<R> outTypeWrapper,
+			StreamInvokable<OUT, R> functionInvokable) {
 		DataStream<OUT> inputStream = this.copy();
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
@@ -1051,14 +1075,14 @@ public class DataStream<OUT> {
 	}
 
 	private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream,
-			SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT> inTypeWrapper) {
+			SinkFunction<OUT> sinkFunction, TypeWrapper<OUT> inTypeWrapper) {
 		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink",
 				outTypeWrapper);
 
 		try {
-			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),
-					inTypeWrapper, null, "sink", SerializationUtils.serialize(sinkFunction),
-					degreeOfParallelism);
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SinkInvokable<OUT>(
+					sinkFunction), inTypeWrapper, null, "sink", SerializationUtils
+					.serialize(sinkFunction), degreeOfParallelism);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize SinkFunction");
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index 4bcdd7b..6bf6f43 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -18,7 +18,7 @@
 package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * Represents the end of a DataStream.
@@ -28,7 +28,7 @@ import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
  */
 public class DataStreamSink<IN> extends SingleOutputStreamOperator<IN, DataStreamSink<IN>> {
 
-	protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType, TypeSerializerWrapper<IN> outTypeWrapper) {
+	protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType, TypeWrapper<IN> outTypeWrapper) {
 		super(environment, operatorType, outTypeWrapper);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
index 5ddc69a..5b2747f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
@@ -18,7 +18,7 @@
 package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * The DataStreamSource represents the starting point of a DataStream.
@@ -28,7 +28,7 @@ import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
  */
 public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataStreamSource<OUT>> {
 
-	public DataStreamSource(StreamExecutionEnvironment environment, String operatorType, TypeSerializerWrapper<OUT> outTypeWrapper) {
+	public DataStreamSource(StreamExecutionEnvironment environment, String operatorType, TypeWrapper<OUT> outTypeWrapper) {
 		super(environment, operatorType, outTypeWrapper);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 676e575..1674c6a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -21,7 +21,7 @@ import org.apache.commons.lang3.SerializationException;
 import org.apache.commons.lang3.SerializationUtils;
 import org.apache.flink.streaming.api.collector.OutputSelector;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * The SingleOutputStreamOperator represents a user defined transformation
@@ -36,7 +36,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 		DataStream<OUT> {
 
 	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment,
-			String operatorType, TypeSerializerWrapper<OUT> outTypeWrapper) {
+			String operatorType, TypeWrapper<OUT> outTypeWrapper) {
 		super(environment, operatorType, outTypeWrapper);
 		setBufferTimeout(environment.getBufferTimeout());
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
new file mode 100644
index 0000000..265e033
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
@@ -0,0 +1,1469 @@
+/**
+ * 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.datastream;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple10;
+import org.apache.flink.api.java.tuple.Tuple11;
+import org.apache.flink.api.java.tuple.Tuple12;
+import org.apache.flink.api.java.tuple.Tuple13;
+import org.apache.flink.api.java.tuple.Tuple14;
+import org.apache.flink.api.java.tuple.Tuple15;
+import org.apache.flink.api.java.tuple.Tuple16;
+import org.apache.flink.api.java.tuple.Tuple17;
+import org.apache.flink.api.java.tuple.Tuple18;
+import org.apache.flink.api.java.tuple.Tuple19;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple20;
+import org.apache.flink.api.java.tuple.Tuple21;
+import org.apache.flink.api.java.tuple.Tuple22;
+import org.apache.flink.api.java.tuple.Tuple23;
+import org.apache.flink.api.java.tuple.Tuple24;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.api.java.tuple.Tuple9;
+import org.apache.flink.streaming.api.invokable.operator.ProjectInvokable;
+import org.apache.flink.streaming.util.serialization.ProjectTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
+
+public class StreamProjection<IN> {
+
+	private DataStream<IN> dataStream;
+	private int[] fieldIndexes;
+	private TypeWrapper<IN> inTypeWrapper;
+
+	protected StreamProjection(DataStream<IN> dataStream, int[] fieldIndexes) {
+		this.dataStream = dataStream;
+		this.fieldIndexes = fieldIndexes;
+		this.inTypeWrapper = dataStream.outTypeWrapper;
+		if (!inTypeWrapper.getTypeInfo().isTupleType()) {
+			throw new RuntimeException("Only Tuple DataStreams can be projected");
+		}
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 * 
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @return The projected DataStream.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0> SingleOutputStreamOperator<Tuple1<T0>, ?> types(Class<T0> type0) {
+		Class<?>[] types = { type0 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple1<T0>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple1<T0>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple1<T0>>(fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 * 
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @return The projected DataStream.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1> SingleOutputStreamOperator<Tuple2<T0, T1>, ?> types(Class<T0> type0,
+			Class<T1> type1) {
+		Class<?>[] types = { type0, type1 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple2<T0, T1>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple2<T0, T1>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple2<T0, T1>>(fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 * 
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @return The projected DataStream.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2> SingleOutputStreamOperator<Tuple3<T0, T1, T2>, ?> types(Class<T0> type0,
+			Class<T1> type1, Class<T2> type2) {
+		Class<?>[] types = { type0, type1, type2 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple3<T0, T1, T2>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple3<T0, T1, T2>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple3<T0, T1, T2>>(fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 * 
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @return The projected DataStream.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3> SingleOutputStreamOperator<Tuple4<T0, T1, T2, T3>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3) {
+		Class<?>[] types = { type0, type1, type2, type3 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple4<T0, T1, T2, T3>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple4<T0, T1, T2, T3>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple4<T0, T1, T2, T3>>(fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 * 
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @return The projected DataStream.
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4> SingleOutputStreamOperator<Tuple5<T0, T1, T2, T3, T4>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4) {
+		Class<?>[] types = { type0, type1, type2, type3, type4 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple5<T0, T1, T2, T3, T4>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple5<T0, T1, T2, T3, T4>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple5<T0, T1, T2, T3, T4>>(fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5> SingleOutputStreamOperator<Tuple6<T0, T1, T2, T3, T4, T5>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple6<T0, T1, T2, T3, T4, T5>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(
+				inTypeWrapper, fieldIndexes, types);
+
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(fieldIndexes,
+						outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6> SingleOutputStreamOperator<Tuple7<T0, T1, T2, T3, T4, T5, T6>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+		TypeWrapper<Tuple7<T0, T1, T2, T3, T4, T5, T6>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fieldIndexes,
+						outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7> SingleOutputStreamOperator<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+		TypeWrapper<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fieldIndexes,
+						outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> SingleOutputStreamOperator<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fieldIndexes,
+						outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> SingleOutputStreamOperator<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(
+						fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> SingleOutputStreamOperator<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+		TypeWrapper<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream.addFunction("projection", null, inTypeWrapper, outTypeWrapper,
+				new ProjectInvokable<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(
+						fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> SingleOutputStreamOperator<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+		TypeWrapper<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(
+								fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> SingleOutputStreamOperator<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> SingleOutputStreamOperator<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(
+								fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> SingleOutputStreamOperator<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(
+								fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> SingleOutputStreamOperator<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(
+								fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> SingleOutputStreamOperator<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> SingleOutputStreamOperator<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> SingleOutputStreamOperator<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> SingleOutputStreamOperator<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(
+								fieldIndexes, outTypeWrapper));
+
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @param type20
+	 *            The class of field '20' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> SingleOutputStreamOperator<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19, Class<T20> type20) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
+				type20 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @param type20
+	 *            The class of field '20' of the result Tuples.
+	 * @param type21
+	 *            The class of field '21' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> SingleOutputStreamOperator<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
+				type20, type21 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @param type20
+	 *            The class of field '20' of the result Tuples.
+	 * @param type21
+	 *            The class of field '21' of the result Tuples.
+	 * @param type22
+	 *            The class of field '22' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> SingleOutputStreamOperator<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
+			Class<T22> type22) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
+				type20, type21, type22 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @param type20
+	 *            The class of field '20' of the result Tuples.
+	 * @param type21
+	 *            The class of field '21' of the result Tuples.
+	 * @param type22
+	 *            The class of field '22' of the result Tuples.
+	 * @param type23
+	 *            The class of field '23' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> SingleOutputStreamOperator<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
+			Class<T22> type22, Class<T23> type23) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
+				type20, type21, type22, type23 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected
+	 * fields. Requires the classes of the fields of the resulting Tuples.
+	 *
+	 * @param type0
+	 *            The class of field '0' of the result Tuples.
+	 * @param type1
+	 *            The class of field '1' of the result Tuples.
+	 * @param type2
+	 *            The class of field '2' of the result Tuples.
+	 * @param type3
+	 *            The class of field '3' of the result Tuples.
+	 * @param type4
+	 *            The class of field '4' of the result Tuples.
+	 * @param type5
+	 *            The class of field '5' of the result Tuples.
+	 * @param type6
+	 *            The class of field '6' of the result Tuples.
+	 * @param type7
+	 *            The class of field '7' of the result Tuples.
+	 * @param type8
+	 *            The class of field '8' of the result Tuples.
+	 * @param type9
+	 *            The class of field '9' of the result Tuples.
+	 * @param type10
+	 *            The class of field '10' of the result Tuples.
+	 * @param type11
+	 *            The class of field '11' of the result Tuples.
+	 * @param type12
+	 *            The class of field '12' of the result Tuples.
+	 * @param type13
+	 *            The class of field '13' of the result Tuples.
+	 * @param type14
+	 *            The class of field '14' of the result Tuples.
+	 * @param type15
+	 *            The class of field '15' of the result Tuples.
+	 * @param type16
+	 *            The class of field '16' of the result Tuples.
+	 * @param type17
+	 *            The class of field '17' of the result Tuples.
+	 * @param type18
+	 *            The class of field '18' of the result Tuples.
+	 * @param type19
+	 *            The class of field '19' of the result Tuples.
+	 * @param type20
+	 *            The class of field '20' of the result Tuples.
+	 * @param type21
+	 *            The class of field '21' of the result Tuples.
+	 * @param type22
+	 *            The class of field '22' of the result Tuples.
+	 * @param type23
+	 *            The class of field '23' of the result Tuples.
+	 * @param type24
+	 *            The class of field '24' of the result Tuples.
+	 * @return The projected DataStream.
+	 *
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24> SingleOutputStreamOperator<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>, ?> types(
+			Class<T0> type0, Class<T1> type1, Class<T2> type2, Class<T3> type3, Class<T4> type4,
+			Class<T5> type5, Class<T6> type6, Class<T7> type7, Class<T8> type8, Class<T9> type9,
+			Class<T10> type10, Class<T11> type11, Class<T12> type12, Class<T13> type13,
+			Class<T14> type14, Class<T15> type15, Class<T16> type16, Class<T17> type17,
+			Class<T18> type18, Class<T19> type19, Class<T20> type20, Class<T21> type21,
+			Class<T22> type22, Class<T23> type23, Class<T24> type24) {
+		Class<?>[] types = { type0, type1, type2, type3, type4, type5, type6, type7, type8, type9,
+				type10, type11, type12, type13, type14, type15, type16, type17, type18, type19,
+				type20, type21, type22, type23, type24 };
+		if (types.length != this.fieldIndexes.length) {
+			throw new IllegalArgumentException(
+					"Numbers of projected fields and types do not match.");
+		}
+
+		TypeWrapper<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> outTypeWrapper = new ProjectTypeWrapper<IN, Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(
+				inTypeWrapper, fieldIndexes, types);
+		return dataStream
+				.addFunction(
+						"projection",
+						null,
+						inTypeWrapper,
+						outTypeWrapper,
+						new ProjectInvokable<IN, Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(
+								fieldIndexes, outTypeWrapper));
+	}
+
+}


[02/12] git commit: [streaming] Fixed InternalStateTest to avoid printouts + minor StreamVertexTest fix

Posted by mb...@apache.org.
[streaming] Fixed InternalStateTest to avoid printouts + minor StreamVertexTest fix


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

Branch: refs/heads/master
Commit: 1bca326f7b66d5ba826bf0783610f25ec3dc7ddd
Parents: 4175dca
Author: Gyula Fora <gy...@apache.org>
Authored: Wed Sep 24 10:46:35 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 .../api/streamvertex/StreamVertexTest.java      |  2 +-
 .../streaming/state/InternalStateTest.java      | 34 +++++++++++++-------
 2 files changed, 23 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1bca326f/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
index 0119f04..e01809d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
@@ -165,7 +165,7 @@ public class StreamVertexTest {
 		fromStringElements.connect(generatedSequence).map(new CoMap()).addSink(new SetSink());
 
 		resultSet = new HashSet<String>();
-		env.execute();
+		env.executeTest(MEMORYSIZE);
 
 		HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
 				"2", "3"));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1bca326f/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
index d215bb4..74299a2 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/InternalStateTest.java
@@ -17,6 +17,12 @@
 
 package org.apache.flink.streaming.state;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.junit.Test;
 
@@ -30,21 +36,25 @@ public class InternalStateTest {
 		state.put("state", "mutable");
 		state.put("streaming", "persist");
 		String s = state.get("streaming");
-		if (s == null) {
-			System.out.println("key does not exist!");
-		} else {
-			System.out.println("value=" + s);
-		}
+		assertEquals("persist", s);
+
 		s = state.get("null");
-		if (s == null) {
-			System.out.println("key does not exist!");
-		} else {
-			System.out.println("value=" + s);
-		}
+
+		assertNull(s);
+
 		TableStateIterator<String, String> iterator = state.getIterator();
+
+		Set<Tuple2<String, String>> expected = new HashSet<Tuple2<String, String>>();
+		expected.add(new Tuple2<String, String>("abc", "hello"));
+		expected.add(new Tuple2<String, String>("test", "world"));
+		expected.add(new Tuple2<String, String>("state", "mutable"));
+		expected.add(new Tuple2<String, String>("streaming", "persist"));
+
+		Set<Tuple2<String, String>> actual = new HashSet<Tuple2<String, String>>();
+
 		while (iterator.hasNext()) {
-			Tuple2<String, String> tuple = iterator.next();
-			System.out.println(tuple.getField(0) + ", " + tuple.getField(1));
+			actual.add(iterator.next());
 		}
+		assertEquals(expected, actual);
 	}
 }


[06/12] [streaming] Streaming jobgraph and vertex refactor to match recent runtime changes

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/InputHandler.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/InputHandler.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/InputHandler.java
deleted file mode 100644
index 0477afa..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/InputHandler.java
+++ /dev/null
@@ -1,104 +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.streamcomponent;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.runtime.io.network.api.MutableReader;
-import org.apache.flink.runtime.io.network.api.MutableRecordReader;
-import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
-import org.apache.flink.runtime.operators.util.ReaderIterator;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.streaming.api.StreamConfig;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
-import org.apache.flink.util.MutableObjectIterator;
-
-public class InputHandler<IN> {
-	private StreamRecordSerializer<IN> inputSerializer = null;
-	private MutableObjectIterator<StreamRecord<IN>> inputIter;
-	private MutableReader<IOReadableWritable> inputs;
-
-	private AbstractStreamComponent streamComponent;
-	private StreamConfig configuration;
-
-	public InputHandler(AbstractStreamComponent streamComponent) {
-		this.streamComponent = streamComponent;
-		this.configuration = new StreamConfig(streamComponent.getTaskConfiguration());
-		try {
-			setConfigInputs();
-		} catch (Exception e) {
-			throw new StreamComponentException("Cannot register inputs for "
-					+ getClass().getSimpleName(), e);
-		}
-
-	}
-
-	@SuppressWarnings("unchecked")
-	protected void setConfigInputs() throws StreamComponentException {
-		setDeserializer();
-
-		int numberOfInputs = configuration.getNumberOfInputs();
-
-		if (numberOfInputs < 2) {
-
-			inputs = new MutableRecordReader<IOReadableWritable>(streamComponent);
-
-		} else {
-			MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
-
-			for (int i = 0; i < numberOfInputs; i++) {
-				recordReaders[i] = new MutableRecordReader<IOReadableWritable>(streamComponent);
-			}
-			inputs = new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
-		}
-
-		inputIter = createInputIterator();
-	}
-
-	private void setDeserializer() {
-		TypeInformation<IN> inTupleTypeInfo = configuration.getTypeInfoIn1();
-		inputSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo);
-	}
-
-	private MutableObjectIterator<StreamRecord<IN>> createInputIterator() {
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(inputs, inputSerializer);
-		return iter;
-	}
-
-	protected static <T> MutableObjectIterator<StreamRecord<T>> staticCreateInputIterator(
-			MutableReader<?> inputReader, TypeSerializer<?> serializer) {
-
-		// generic data type serialization
-		@SuppressWarnings("unchecked")
-		MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		final MutableObjectIterator<StreamRecord<T>> iter = new ReaderIterator(reader, serializer);
-		return iter;
-	}
-
-	public StreamRecordSerializer<IN> getInputSerializer() {
-		return inputSerializer;
-	}
-
-	public MutableObjectIterator<StreamRecord<IN>> getInputIter() {
-		return inputIter;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/OutputHandler.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/OutputHandler.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/OutputHandler.java
deleted file mode 100644
index dabb871..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/OutputHandler.java
+++ /dev/null
@@ -1,179 +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.streamcomponent;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.StreamConfig;
-import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
-import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.collector.StreamCollector;
-import org.apache.flink.streaming.api.invokable.StreamInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
-import org.apache.flink.streaming.io.StreamRecordWriter;
-import org.apache.flink.streaming.partitioner.StreamPartitioner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class OutputHandler<OUT> {
-	private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class);
-
-	private AbstractStreamComponent streamComponent;
-	private StreamConfig configuration;
-
-	private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
-	private StreamCollector<OUT> collector;
-	private long bufferTimeout;
-
-	TypeInformation<OUT> outTypeInfo = null;
-	StreamRecordSerializer<OUT> outSerializer = null;
-	SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
-
-	public OutputHandler(AbstractStreamComponent streamComponent) {
-		this.streamComponent = streamComponent;
-		this.outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
-		this.configuration = new StreamConfig(streamComponent.getTaskConfiguration());
-
-		try {
-			setConfigOutputs();
-		} catch (StreamComponentException e) {
-			throw new StreamComponentException("Cannot register outputs for "
-					+ streamComponent.getClass().getSimpleName(), e);
-		}
-	}
-
-	public List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> getOutputs() {
-		return outputs;
-	}
-
-	private void setConfigOutputs() {
-		setSerializers();
-		setCollector();
-
-		int numberOfOutputs = configuration.getNumberOfOutputs();
-		bufferTimeout = configuration.getBufferTimeout();
-
-		for (int i = 0; i < numberOfOutputs; i++) {
-			setPartitioner(i, outputs);
-		}
-	}
-
-	private StreamCollector<OUT> setCollector() {
-		if (streamComponent.configuration.getDirectedEmit()) {
-			OutputSelector<OUT> outputSelector = streamComponent.configuration.getOutputSelector();
-
-			collector = new DirectedStreamCollector<OUT>(streamComponent.getInstanceID(),
-					outSerializationDelegate, outputSelector);
-		} else {
-			collector = new StreamCollector<OUT>(streamComponent.getInstanceID(),
-					outSerializationDelegate);
-		}
-		return collector;
-	}
-
-	public StreamCollector<OUT> getCollector() {
-		return collector;
-	}
-
-	void setSerializers() {
-		outTypeInfo = configuration.getTypeInfoOut1();
-		outSerializer = new StreamRecordSerializer<OUT>(outTypeInfo);
-		outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outSerializer);
-		outSerializationDelegate.setInstance(outSerializer.createInstance());
-	}
-
-	void setPartitioner(int outputNumber,
-			List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
-		StreamPartitioner<OUT> outputPartitioner = null;
-
-		try {
-			outputPartitioner = configuration.getPartitioner(outputNumber);
-
-		} catch (Exception e) {
-			throw new StreamComponentException("Cannot deserialize partitioner for "
-					+ streamComponent.getName() + " with " + outputNumber + " outputs", e);
-		}
-
-		RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
-
-		if (bufferTimeout > 0) {
-			output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(
-					streamComponent, outputPartitioner, bufferTimeout);
-		} else {
-			output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(streamComponent,
-					outputPartitioner);
-		}
-
-		outputs.add(output);
-		List<String> outputName = configuration.getOutputName(outputNumber);
-		boolean isSelectAllOutput = configuration.getSelectAll(outputNumber);
-
-		if (collector != null) {
-			collector.addOutput(output, outputName, isSelectAllOutput);
-		}
-
-		if (LOG.isTraceEnabled()) {
-			LOG.trace("Partitioner set: {} with {} outputs", outputPartitioner.getClass()
-					.getSimpleName(), outputNumber);
-		}
-	}
-
-	public void flushOutputs() throws IOException, InterruptedException {
-		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
-			output.flush();
-		}
-	}
-
-	public void initializeOutputSerializers() {
-		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
-			output.initializeSerializers();
-		}
-	}
-
-	long startTime;
-
-	public void invokeUserFunction(String componentTypeName, StreamInvokable<OUT> userInvokable)
-			throws IOException, InterruptedException {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{} {} invoked with instance id {}", componentTypeName,
-					streamComponent.getName(), streamComponent.getInstanceID());
-		}
-
-		initializeOutputSerializers();
-
-		try {
-			streamComponent.invokeUserFunction(userInvokable);
-		} catch (Exception e) {
-			flushOutputs();
-			throw new RuntimeException(e);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{} {} invoke finished instance id {}", componentTypeName,
-					streamComponent.getName(), streamComponent.getInstanceID());
-		}
-
-		flushOutputs();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentException.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentException.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentException.java
deleted file mode 100644
index 240c9ba..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentException.java
+++ /dev/null
@@ -1,68 +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.streamcomponent;
-
-/**
- * An exception that is thrown by the stream components when encountering an
- * illegal condition.
- */
-public class StreamComponentException extends RuntimeException {
-
-	/**
-	 * Serial version UID for serialization interoperability.
-	 */
-	private static final long serialVersionUID = 8392043527067472439L;
-
-	/**
-	 * Creates a compiler exception with no message and no cause.
-	 */
-	public StreamComponentException() {
-	}
-
-	/**
-	 * Creates a compiler exception with the given message and no cause.
-	 * 
-	 * @param message
-	 *            The message for the exception.
-	 */
-	public StreamComponentException(String message) {
-		super(message);
-	}
-
-	/**
-	 * Creates a compiler exception with the given cause and no message.
-	 * 
-	 * @param cause
-	 *            The <tt>Throwable</tt> that caused this exception.
-	 */
-	public StreamComponentException(Throwable cause) {
-		super(cause);
-	}
-
-	/**
-	 * Creates a compiler exception with the given message and cause.
-	 * 
-	 * @param message
-	 *            The message for the exception.
-	 * @param cause
-	 *            The <tt>Throwable</tt> that caused this exception.
-	 */
-	public StreamComponentException(String message, Throwable cause) {
-		super(message, cause);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
deleted file mode 100755
index 32fc5f9..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSink.java
+++ /dev/null
@@ -1,104 +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.streamcomponent;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.io.BlockingQueueBroker;
-import org.apache.flink.util.StringUtils;
-
-public class StreamIterationSink<IN extends Tuple> extends AbstractStreamComponent {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationSink.class);
-
-	private InputHandler<IN> inputHandler;
-
-	private String iterationId;
-	@SuppressWarnings("rawtypes")
-	private BlockingQueue<StreamRecord> dataChannel;
-	private long iterationWaitTime;
-	private boolean shouldWait;
-
-	public StreamIterationSink() {
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		try {
-			inputHandler = new InputHandler<IN>(this);
-
-			iterationId = configuration.getIterationId();
-			iterationWaitTime = configuration.getIterationWaitTime();
-			shouldWait = iterationWaitTime > 0;
-			dataChannel = BlockingQueueBroker.instance().get(iterationId);
-		} catch (Exception e) {
-			throw new StreamComponentException(String.format(
-					"Cannot register inputs of StreamIterationSink %s", iterationId), e);
-		}
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK {} invoked", getName());
-		}
-
-		forwardRecords();
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK {} invoke finished", getName());
-		}
-	}
-
-	protected void forwardRecords() throws Exception {
-		StreamRecord<IN> reuse = inputHandler.getInputSerializer().createInstance();
-		while ((reuse = inputHandler.getInputIter().next(reuse)) != null) {
-			if (!pushToQueue(reuse)) {
-				break;
-			}
-			// TODO: Fix object reuse for iteration
-			reuse = inputHandler.getInputSerializer().createInstance();
-		}
-	}
-
-	private boolean pushToQueue(StreamRecord<IN> record) {
-		try {
-			if (shouldWait) {
-				return dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS);
-			} else {
-				dataChannel.put(record);
-				return true;
-			}
-		} catch (InterruptedException e) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Pushing back record at iteration %s failed due to: {}", iterationId,
-						StringUtils.stringifyException(e));
-			}
-			return false;
-		}
-	}
-
-	@Override
-	protected void setInvokable() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
deleted file mode 100755
index ce5687a..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamIterationSource.java
+++ /dev/null
@@ -1,100 +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.streamcomponent;
-
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.io.BlockingQueueBroker;
-
-public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComponent {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationSource.class);
-
-	private OutputHandler<OUT> outputHandler;
-
-	private static int numSources;
-	private String iterationId;
-	@SuppressWarnings("rawtypes")
-	private BlockingQueue<StreamRecord> dataChannel;
-	private long iterationWaitTime;
-	private boolean shouldWait;
-
-	@SuppressWarnings("rawtypes")
-	public StreamIterationSource() {
-		numSources = newComponent();
-		instanceID = numSources;
-		dataChannel = new ArrayBlockingQueue<StreamRecord>(1);
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		outputHandler = new OutputHandler<OUT>(this);
-
-		iterationId = configuration.getIterationId();
-		iterationWaitTime = configuration.getIterationWaitTime();
-		shouldWait = iterationWaitTime > 0;
-
-		try {
-			BlockingQueueBroker.instance().handIn(iterationId, dataChannel);
-		} catch (Exception e) {
-
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void invoke() throws Exception {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SOURCE {} invoked with instance id {}", getName(), getInstanceID());
-		}
-
-		outputHandler.initializeOutputSerializers();
-
-		StreamRecord<OUT> nextRecord;
-
-		while (true) {
-			if (shouldWait) {
-				nextRecord = dataChannel.poll(iterationWaitTime, TimeUnit.MILLISECONDS);
-			} else {
-				nextRecord = dataChannel.take();
-			}
-			if (nextRecord == null) {
-				break;
-			}
-			for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputHandler
-					.getOutputs()) {
-				outputHandler.outSerializationDelegate.setInstance(nextRecord);
-				output.emit(outputHandler.outSerializationDelegate);
-			}
-		}
-
-		outputHandler.flushOutputs();
-	}
-
-	@Override
-	protected void setInvokable() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
deleted file mode 100644
index e2982bf..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSink.java
+++ /dev/null
@@ -1,61 +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.streamcomponent;
-
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamSink<IN> extends AbstractStreamComponent {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamSink.class);
-
-	private InputHandler<IN> inputHandler;
-	
-	private StreamOperatorInvokable<IN, IN> userInvokable;
-
-	public StreamSink() {
-		userInvokable = null;
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		inputHandler = new InputHandler<IN>(this);
-	}
-
-	@Override
-	protected void setInvokable() {
-		userInvokable = configuration.getUserInvokable();
-		userInvokable.initialize(null, inputHandler.getInputIter(), inputHandler.getInputSerializer(),
-				isMutable);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK {} invoked", getName());
-		}
-
-		invokeUserFunction(userInvokable);
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK {} invoke finished", getName());
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
deleted file mode 100644
index 11f372a..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamSource.java
+++ /dev/null
@@ -1,52 +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.streamcomponent;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.invokable.SourceInvokable;
-
-public class StreamSource<OUT extends Tuple> extends AbstractStreamComponent {
-
-	protected OutputHandler<OUT> outputHandler;
-
-	private SourceInvokable<OUT> sourceInvokable;
-	
-	private static int numSources;
-
-	public StreamSource() {
-		sourceInvokable = null;
-		numSources = newComponent();
-		instanceID = numSources;
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		outputHandler = new OutputHandler<OUT>(this);
-	}
-
-	@Override
-	protected void setInvokable() {
-		sourceInvokable = configuration.getUserInvokable();
-		sourceInvokable.setCollector(outputHandler.getCollector());
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		outputHandler.invokeUserFunction("SOURCE", sourceInvokable);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.java
deleted file mode 100644
index 6824d09..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/StreamTask.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.api.streamcomponent;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
-
-public class StreamTask<IN extends Tuple, OUT extends Tuple> extends AbstractStreamComponent {
-
-	private InputHandler<IN> inputHandler;
-	private OutputHandler<OUT> outputHandler;
-
-	private StreamOperatorInvokable<IN, OUT> userInvokable;
-	
-	private static int numTasks;
-
-	public StreamTask() {
-		userInvokable = null;
-		numTasks = newComponent();
-		instanceID = numTasks;
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		inputHandler = new InputHandler<IN>(this);
-		outputHandler = new OutputHandler<OUT>(this);
-	}
-
-	@Override
-	protected void setInvokable() {
-		userInvokable = configuration.getUserInvokable();
-		userInvokable.initialize(outputHandler.getCollector(), inputHandler.getInputIter(),
-				inputHandler.getInputSerializer(), isMutable);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		outputHandler.invokeUserFunction("TASK", userInvokable);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java
new file mode 100644
index 0000000..5a6519d
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java
@@ -0,0 +1,113 @@
+/*
+ * 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.streamvertex;
+
+import java.util.ArrayList;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.io.network.api.MutableRecordReader;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.streaming.io.CoReaderIterator;
+import org.apache.flink.streaming.io.CoRecordReader;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class CoStreamVertex<IN1, IN2, OUT> extends
+		StreamVertex<IN1,OUT> {
+
+	private OutputHandler<OUT> outputHandler;
+
+	protected StreamRecordSerializer<IN1> inputDeserializer1 = null;
+	protected StreamRecordSerializer<IN2> inputDeserializer2 = null;
+
+	MutableObjectIterator<StreamRecord<IN1>> inputIter1;
+	MutableObjectIterator<StreamRecord<IN2>> inputIter2;
+
+	CoRecordReader<DeserializationDelegate<StreamRecord<IN1>>, DeserializationDelegate<StreamRecord<IN2>>> coReader;
+	CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> coIter;
+
+	private CoInvokable<IN1, IN2, OUT> userInvokable;
+	private static int numTasks;
+
+	public CoStreamVertex() {
+		userInvokable = null;
+		numTasks = newVertex();
+		instanceID = numTasks;
+	}
+
+	private void setDeserializers() {
+		TypeInformation<IN1> inputTypeInfo1 = configuration.getTypeInfoIn1();
+		inputDeserializer1 = new StreamRecordSerializer<IN1>(inputTypeInfo1);
+
+		TypeInformation<IN2> inputTypeInfo2 = configuration.getTypeInfoIn2();
+		inputDeserializer2 = new StreamRecordSerializer<IN2>(inputTypeInfo2);
+	}
+
+	@Override
+	public void setInputsOutputs() {
+		outputHandler = new OutputHandler<OUT>(this);
+
+		setConfigInputs();
+
+		coIter = new CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>>(coReader,
+				inputDeserializer1, inputDeserializer2);
+	}
+
+	@Override
+	protected void setInvokable() {
+		userInvokable = configuration.getUserInvokable();
+		userInvokable.initialize(outputHandler.getCollector(), coIter, inputDeserializer1,
+				inputDeserializer2, isMutable);
+	}
+
+	protected void setConfigInputs() throws StreamVertexException {
+		setDeserializers();
+
+		int numberOfInputs = configuration.getNumberOfInputs();
+
+		ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>> inputList1 = new ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>>();
+		ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>> inputList2 = new ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>>();
+
+		for (int i = 0; i < numberOfInputs; i++) {
+			int inputType = configuration.getInputType(i);
+			switch (inputType) {
+			case 1:
+				inputList1.add(new MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>(
+						this));
+				break;
+			case 2:
+				inputList2.add(new MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>(
+						this));
+				break;
+			default:
+				throw new RuntimeException("Invalid input type number: " + inputType);
+			}
+		}
+
+		coReader = new CoRecordReader<DeserializationDelegate<StreamRecord<IN1>>, DeserializationDelegate<StreamRecord<IN2>>>(
+				inputList1, inputList2);
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		outputHandler.invokeUserFunction("CO-TASK", userInvokable);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java
new file mode 100644
index 0000000..17d2ae5
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/InputHandler.java
@@ -0,0 +1,109 @@
+/*
+ * 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.streamvertex;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.io.network.api.MutableReader;
+import org.apache.flink.runtime.io.network.api.MutableRecordReader;
+import org.apache.flink.runtime.io.network.api.MutableUnionRecordReader;
+import org.apache.flink.runtime.operators.util.ReaderIterator;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.streaming.api.StreamConfig;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.MutableObjectIterator;
+
+public class InputHandler<IN> {
+	private StreamRecordSerializer<IN> inputSerializer = null;
+	private MutableObjectIterator<StreamRecord<IN>> inputIter;
+	private MutableReader<IOReadableWritable> inputs;
+
+	private StreamVertex<IN,?> streamVertex;
+	private StreamConfig configuration;
+
+	public InputHandler(StreamVertex<IN,?> streamComponent) {
+		this.streamVertex = streamComponent;
+		this.configuration = new StreamConfig(streamComponent.getTaskConfiguration());
+		try {
+			setConfigInputs();
+		} catch (Exception e) {
+			throw new StreamVertexException("Cannot register inputs for "
+					+ getClass().getSimpleName(), e);
+		}
+
+	}
+
+	@SuppressWarnings("unchecked")
+	protected void setConfigInputs() throws StreamVertexException {
+		setDeserializer();
+
+		int numberOfInputs = configuration.getNumberOfInputs();
+		if (numberOfInputs > 0) {
+
+			if (numberOfInputs < 2) {
+
+				inputs = new MutableRecordReader<IOReadableWritable>(streamVertex);
+
+			} else {
+				MutableRecordReader<IOReadableWritable>[] recordReaders = (MutableRecordReader<IOReadableWritable>[]) new MutableRecordReader<?>[numberOfInputs];
+
+				for (int i = 0; i < numberOfInputs; i++) {
+					recordReaders[i] = new MutableRecordReader<IOReadableWritable>(streamVertex);
+				}
+				inputs = new MutableUnionRecordReader<IOReadableWritable>(recordReaders);
+			}
+
+			inputIter = createInputIterator();
+		}
+	}
+
+	private void setDeserializer() {
+		TypeInformation<IN> inTupleTypeInfo = configuration.getTypeInfoIn1();
+		if (inTupleTypeInfo != null) {
+			inputSerializer = new StreamRecordSerializer<IN>(inTupleTypeInfo);
+		}
+	}
+
+	private MutableObjectIterator<StreamRecord<IN>> createInputIterator() {
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		final MutableObjectIterator<StreamRecord<IN>> iter = new ReaderIterator(inputs,
+				inputSerializer);
+		return iter;
+	}
+
+	protected static <T> MutableObjectIterator<StreamRecord<T>> staticCreateInputIterator(
+			MutableReader<?> inputReader, TypeSerializer<?> serializer) {
+
+		// generic data type serialization
+		@SuppressWarnings("unchecked")
+		MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		final MutableObjectIterator<StreamRecord<T>> iter = new ReaderIterator(reader, serializer);
+		return iter;
+	}
+
+	public StreamRecordSerializer<IN> getInputSerializer() {
+		return inputSerializer;
+	}
+
+	public MutableObjectIterator<StreamRecord<IN>> getInputIter() {
+		return inputIter;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java
new file mode 100644
index 0000000..d3f75dd
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java
@@ -0,0 +1,181 @@
+/*
+ * 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.streamvertex;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.StreamConfig;
+import org.apache.flink.streaming.api.collector.DirectedStreamCollector;
+import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.streaming.api.collector.StreamCollector;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
+import org.apache.flink.streaming.io.StreamRecordWriter;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OutputHandler<OUT> {
+	private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class);
+
+	private StreamVertex<?,OUT> streamVertex;
+	private StreamConfig configuration;
+
+	private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
+	private StreamCollector<OUT> collector;
+	private long bufferTimeout;
+
+	TypeInformation<OUT> outTypeInfo = null;
+	StreamRecordSerializer<OUT> outSerializer = null;
+	SerializationDelegate<StreamRecord<OUT>> outSerializationDelegate = null;
+
+	public OutputHandler(StreamVertex<?,OUT> streamComponent) {
+		this.streamVertex = streamComponent;
+		this.outputs = new LinkedList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+		this.configuration = new StreamConfig(streamComponent.getTaskConfiguration());
+
+		try {
+			setConfigOutputs();
+		} catch (StreamVertexException e) {
+			throw new StreamVertexException("Cannot register outputs for "
+					+ streamComponent.getClass().getSimpleName(), e);
+		}
+	}
+
+	public List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> getOutputs() {
+		return outputs;
+	}
+
+	private void setConfigOutputs() {
+		setSerializers();
+		setCollector();
+
+		int numberOfOutputs = configuration.getNumberOfOutputs();
+		bufferTimeout = configuration.getBufferTimeout();
+
+		for (int i = 0; i < numberOfOutputs; i++) {
+			setPartitioner(i, outputs);
+		}
+	}
+
+	private StreamCollector<OUT> setCollector() {
+		if (streamVertex.configuration.getDirectedEmit()) {
+			OutputSelector<OUT> outputSelector = streamVertex.configuration.getOutputSelector();
+
+			collector = new DirectedStreamCollector<OUT>(streamVertex.getInstanceID(),
+					outSerializationDelegate, outputSelector);
+		} else {
+			collector = new StreamCollector<OUT>(streamVertex.getInstanceID(),
+					outSerializationDelegate);
+		}
+		return collector;
+	}
+
+	public StreamCollector<OUT> getCollector() {
+		return collector;
+	}
+
+	void setSerializers() {
+		outTypeInfo = configuration.getTypeInfoOut1();
+		if (outTypeInfo != null) {
+			outSerializer = new StreamRecordSerializer<OUT>(outTypeInfo);
+			outSerializationDelegate = new SerializationDelegate<StreamRecord<OUT>>(outSerializer);
+			outSerializationDelegate.setInstance(outSerializer.createInstance());
+		}
+	}
+
+	void setPartitioner(int outputNumber,
+			List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs) {
+		StreamPartitioner<OUT> outputPartitioner = null;
+
+		try {
+			outputPartitioner = configuration.getPartitioner(outputNumber);
+
+		} catch (Exception e) {
+			throw new StreamVertexException("Cannot deserialize partitioner for "
+					+ streamVertex.getName() + " with " + outputNumber + " outputs", e);
+		}
+
+		RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output;
+
+		if (bufferTimeout > 0) {
+			output = new StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>(
+					streamVertex, outputPartitioner, bufferTimeout);
+		} else {
+			output = new RecordWriter<SerializationDelegate<StreamRecord<OUT>>>(streamVertex,
+					outputPartitioner);
+		}
+
+		outputs.add(output);
+		List<String> outputName = configuration.getOutputName(outputNumber);
+		boolean isSelectAllOutput = configuration.getSelectAll(outputNumber);
+
+		if (collector != null) {
+			collector.addOutput(output, outputName, isSelectAllOutput);
+		}
+
+		if (LOG.isTraceEnabled()) {
+			LOG.trace("Partitioner set: {} with {} outputs", outputPartitioner.getClass()
+					.getSimpleName(), outputNumber);
+		}
+	}
+
+	public void flushOutputs() throws IOException, InterruptedException {
+		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+			output.flush();
+		}
+	}
+
+	public void initializeOutputSerializers() {
+		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+			output.initializeSerializers();
+		}
+	}
+
+	long startTime;
+
+	public void invokeUserFunction(String componentTypeName, StreamInvokable<?,OUT> userInvokable)
+			throws IOException, InterruptedException {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("{} {} invoked with instance id {}", componentTypeName,
+					streamVertex.getName(), streamVertex.getInstanceID());
+		}
+
+		initializeOutputSerializers();
+
+		try {
+			streamVertex.invokeUserFunction(userInvokable);
+		} catch (Exception e) {
+			flushOutputs();
+			throw new RuntimeException(e);
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("{} {} invoke finished instance id {}", componentTypeName,
+					streamVertex.getName(), streamVertex.getInstanceID());
+		}
+
+		flushOutputs();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationHead.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationHead.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationHead.java
new file mode 100755
index 0000000..4dfecb1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationHead.java
@@ -0,0 +1,100 @@
+/*
+ * 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.streamvertex;
+
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.io.BlockingQueueBroker;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StreamIterationHead<OUT extends Tuple> extends StreamVertex<OUT,OUT> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationHead.class);
+
+	private OutputHandler<OUT> outputHandler;
+
+	private static int numSources;
+	private String iterationId;
+	@SuppressWarnings("rawtypes")
+	private BlockingQueue<StreamRecord> dataChannel;
+	private long iterationWaitTime;
+	private boolean shouldWait;
+
+	@SuppressWarnings("rawtypes")
+	public StreamIterationHead() {
+		numSources = newVertex();
+		instanceID = numSources;
+		dataChannel = new ArrayBlockingQueue<StreamRecord>(1);
+	}
+
+	@Override
+	public void setInputsOutputs() {
+		outputHandler = new OutputHandler<OUT>(this);
+
+		iterationId = configuration.getIterationId();
+		iterationWaitTime = configuration.getIterationWaitTime();
+		shouldWait = iterationWaitTime > 0;
+
+		try {
+			BlockingQueueBroker.instance().handIn(iterationId, dataChannel);
+		} catch (Exception e) {
+
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void invoke() throws Exception {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("SOURCE {} invoked with instance id {}", getName(), getInstanceID());
+		}
+
+		outputHandler.initializeOutputSerializers();
+
+		StreamRecord<OUT> nextRecord;
+
+		while (true) {
+			if (shouldWait) {
+				nextRecord = dataChannel.poll(iterationWaitTime, TimeUnit.MILLISECONDS);
+			} else {
+				nextRecord = dataChannel.take();
+			}
+			if (nextRecord == null) {
+				break;
+			}
+			for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputHandler
+					.getOutputs()) {
+				outputHandler.outSerializationDelegate.setInstance(nextRecord);
+				output.emit(outputHandler.outSerializationDelegate);
+			}
+		}
+
+		outputHandler.flushOutputs();
+	}
+
+	@Override
+	protected void setInvokable() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationTail.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationTail.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationTail.java
new file mode 100755
index 0000000..b603686
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamIterationTail.java
@@ -0,0 +1,104 @@
+/*
+ * 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.streamvertex;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.io.BlockingQueueBroker;
+import org.apache.flink.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StreamIterationTail<IN extends Tuple> extends StreamVertex<IN,IN> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationTail.class);
+
+	private InputHandler<IN> inputHandler;
+
+	private String iterationId;
+	@SuppressWarnings("rawtypes")
+	private BlockingQueue<StreamRecord> dataChannel;
+	private long iterationWaitTime;
+	private boolean shouldWait;
+
+	public StreamIterationTail() {
+	}
+
+	@Override
+	public void setInputsOutputs() {
+		try {
+			inputHandler = new InputHandler<IN>(this);
+
+			iterationId = configuration.getIterationId();
+			iterationWaitTime = configuration.getIterationWaitTime();
+			shouldWait = iterationWaitTime > 0;
+			dataChannel = BlockingQueueBroker.instance().get(iterationId);
+		} catch (Exception e) {
+			throw new StreamVertexException(String.format(
+					"Cannot register inputs of StreamIterationSink %s", iterationId), e);
+		}
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("SINK {} invoked", getName());
+		}
+
+		forwardRecords();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("SINK {} invoke finished", getName());
+		}
+	}
+
+	protected void forwardRecords() throws Exception {
+		StreamRecord<IN> reuse = inputHandler.getInputSerializer().createInstance();
+		while ((reuse = inputHandler.getInputIter().next(reuse)) != null) {
+			if (!pushToQueue(reuse)) {
+				break;
+			}
+			// TODO: Fix object reuse for iteration
+			reuse = inputHandler.getInputSerializer().createInstance();
+		}
+	}
+
+	private boolean pushToQueue(StreamRecord<IN> record) {
+		try {
+			if (shouldWait) {
+				return dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS);
+			} else {
+				dataChannel.put(record);
+				return true;
+			}
+		} catch (InterruptedException e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Pushing back record at iteration %s failed due to: {}", iterationId,
+						StringUtils.stringifyException(e));
+			}
+			return false;
+		}
+	}
+
+	@Override
+	protected void setInvokable() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java
new file mode 100644
index 0000000..d05339a
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java
@@ -0,0 +1,96 @@
+/*
+ * 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.streamvertex;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.streaming.api.StreamConfig;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
+
+public class StreamVertex<IN, OUT> extends AbstractInvokable {
+	
+	private static int numTasks;
+	
+	protected StreamConfig configuration;
+	protected int instanceID;
+	protected String name;
+	private static int numVertices = 0;
+	protected boolean isMutable;
+	protected Object function;
+	protected String functionName;
+	
+	private InputHandler<IN> inputHandler;
+	private OutputHandler<OUT> outputHandler;
+	private StreamInvokable<IN, OUT> userInvokable;
+	
+	public StreamVertex() {
+		userInvokable = null;
+		numTasks = newVertex();
+		instanceID = numTasks;
+	}
+
+	protected static int newVertex() {
+		numVertices++;
+		return numVertices;
+	}
+
+	@Override
+	public void registerInputOutput() {
+		initialize();
+		setInputsOutputs();
+		setInvokable();
+	}
+	
+	protected void initialize() {
+		this.configuration = new StreamConfig(getTaskConfiguration());
+		this.name = configuration.getVertexName();
+		this.isMutable = configuration.getMutability();
+		this.functionName = configuration.getFunctionName();
+		this.function = configuration.getFunction();
+	}
+
+	protected <T> void invokeUserFunction(StreamInvokable<?,T> userInvokable) throws Exception {
+		userInvokable.open(getTaskConfiguration());
+		userInvokable.invoke();
+		userInvokable.close();
+	}
+	
+
+	public void setInputsOutputs() {
+		inputHandler = new InputHandler<IN>(this);
+		outputHandler = new OutputHandler<OUT>(this);
+	}
+
+	protected void setInvokable() {
+		userInvokable = configuration.getUserInvokable();
+		userInvokable.initialize(outputHandler.getCollector(), inputHandler.getInputIter(),
+				inputHandler.getInputSerializer(), isMutable);
+	}
+	
+	public String getName() {
+		return name;
+	}
+
+	public int getInstanceID() {
+		return instanceID;
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		outputHandler.invokeUserFunction("TASK", userInvokable);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertexException.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertexException.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertexException.java
new file mode 100644
index 0000000..ed8b91e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertexException.java
@@ -0,0 +1,68 @@
+/*
+ * 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.streamvertex;
+
+/**
+ * An exception that is thrown by the stream verices when encountering an
+ * illegal condition.
+ */
+public class StreamVertexException extends RuntimeException {
+
+	/**
+	 * Serial version UID for serialization interoperability.
+	 */
+	private static final long serialVersionUID = 8392043527067472439L;
+
+	/**
+	 * Creates a compiler exception with no message and no cause.
+	 */
+	public StreamVertexException() {
+	}
+
+	/**
+	 * Creates a compiler exception with the given message and no cause.
+	 * 
+	 * @param message
+	 *            The message for the exception.
+	 */
+	public StreamVertexException(String message) {
+		super(message);
+	}
+
+	/**
+	 * Creates a compiler exception with the given cause and no message.
+	 * 
+	 * @param cause
+	 *            The <tt>Throwable</tt> that caused this exception.
+	 */
+	public StreamVertexException(Throwable cause) {
+		super(cause);
+	}
+
+	/**
+	 * Creates a compiler exception with the given message and cause.
+	 * 
+	 * @param message
+	 *            The message for the exception.
+	 * @param cause
+	 *            The <tt>Throwable</tt> that caused this exception.
+	 */
+	public StreamVertexException(String message, Throwable cause) {
+		super(message, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
index ae52e67..535e109 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/StreamCollectorTest.java
@@ -18,13 +18,13 @@
 package org.apache.flink.streaming.api.collector;
 
 import static org.junit.Assert.assertArrayEquals;
-
+
 import java.util.ArrayList;
-
+
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.streamcomponent.MockRecordWriter;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamvertex.MockRecordWriter;
 import org.apache.flink.streaming.util.MockRecordWriterFactory;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
deleted file mode 100755
index 9c5c43f..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/MockRecordWriter.java
+++ /dev/null
@@ -1,45 +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.streamcomponent;
-
-import java.util.ArrayList;
-
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.runtime.operators.DataSourceTask;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-
-public class MockRecordWriter extends RecordWriter<SerializationDelegate<StreamRecord<Tuple1<Integer>>>> {
-
-	public ArrayList<Integer> emittedRecords;
-
-	public MockRecordWriter(DataSourceTask<?> inputBase, Class<StreamRecord<Tuple1<Integer>>> outputClass) {
-		super(inputBase);
-	}
-
-	public boolean initList() {
-		emittedRecords = new ArrayList<Integer>();
-		return true;
-	}
-	
-	@Override
-	public void emit(SerializationDelegate<StreamRecord<Tuple1<Integer>>> record) {
-		emittedRecords.add(record.getInstance().getObject().f0);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
deleted file mode 100644
index cc341dc..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ /dev/null
@@ -1,197 +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.streamcomponent;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.co.CoMapFunction;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
-import org.apache.flink.streaming.api.function.source.SourceFunction;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class StreamComponentTest {
-
-	private static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
-
-	public static class MySource implements SourceFunction<Tuple1<Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		private Tuple1<Integer> tuple = new Tuple1<Integer>(0);
-
-		@Override
-		public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
-			for (int i = 0; i < 10; i++) {
-				tuple.f0 = i;
-				collector.collect(tuple);
-			}
-		}
-	}
-
-	public static class MyTask extends RichMapFunction<Tuple1<Integer>, Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> map(Tuple1<Integer> value) throws Exception {
-			Integer i = value.f0;
-			return new Tuple2<Integer, Integer>(i, i + 1);
-		}
-	}
-
-	public static class MySink implements SinkFunction<Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void invoke(Tuple2<Integer, Integer> tuple) {
-			Integer k = tuple.getField(0);
-			Integer v = tuple.getField(1);
-			data.put(k, v);
-		}
-	}
-
-	@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);
-
-		try {
-			env.execute();
-			fail();
-		} catch (Exception e) {
-		}
-
-		env.fromCollection(Arrays.asList("a", "b"));
-
-		try {
-			env.execute();
-			fail();
-		} catch (Exception e) {
-		}
-
-		try {
-			env.fromCollection(null);
-			fail();
-		} catch (NullPointerException e) {
-		}
-
-		try {
-			env.fromElements();
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.generateSequence(-10, -30);
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.setBufferTimeout(-10);
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.setExecutionParallelism(-10);
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.readTextFile("random/path/that/is/not/valid");
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-	}
-
-	private static class CoMap implements CoMapFunction<String, Long, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map1(String value) {
-			return value;
-		}
-
-		@Override
-		public String map2(Long value) {
-			return value.toString();
-		}
-	}
-
-	static HashSet<String> resultSet;
-
-	private static class SetSink implements SinkFunction<String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void invoke(String value) {
-			resultSet.add(value);
-		}
-	}
-
-	@Test
-	public void coTest() throws Exception {
-		LocalStreamEnvironment env = StreamExecutionEnvironment
-				.createLocalEnvironment(SOURCE_PARALELISM);
-
-		DataStream<String> fromStringElements = env.fromElements("aa", "bb", "cc");
-		DataStream<Long> generatedSequence = env.generateSequence(0, 3);
-
-		fromStringElements.connect(generatedSequence).map(new CoMap()).addSink(new SetSink());
-
-		resultSet = new HashSet<String>();
-		env.execute();
-
-		HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
-				"2", "3"));
-		assertEquals(expectedSet, resultSet);
-	}
-
-	@Test
-	public void runStream() throws Exception {
-		LocalStreamEnvironment env = StreamExecutionEnvironment
-				.createLocalEnvironment(SOURCE_PARALELISM);
-
-		env.addSource(new MySource(), SOURCE_PARALELISM).map(new MyTask()).addSink(new MySink());
-
-		env.executeTest(MEMORYSIZE);
-		assertEquals(10, data.keySet().size());
-
-		for (Integer k : data.keySet()) {
-			assertEquals((Integer) (k + 1), data.get(k));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/MockRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/MockRecordWriter.java
new file mode 100755
index 0000000..e91dc8b
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/MockRecordWriter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.streamvertex;
+
+import java.util.ArrayList;
+
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.runtime.operators.DataSourceTask;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+
+public class MockRecordWriter extends RecordWriter<SerializationDelegate<StreamRecord<Tuple1<Integer>>>> {
+
+	public ArrayList<Integer> emittedRecords;
+
+	public MockRecordWriter(DataSourceTask<?> inputBase, Class<StreamRecord<Tuple1<Integer>>> outputClass) {
+		super(inputBase);
+	}
+
+	public boolean initList() {
+		emittedRecords = new ArrayList<Integer>();
+		return true;
+	}
+	
+	@Override
+	public void emit(SerializationDelegate<StreamRecord<Tuple1<Integer>>> record) {
+		emittedRecords.add(record.getInstance().getObject().f0);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
new file mode 100644
index 0000000..f470c76
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.streamvertex;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.co.CoMapFunction;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.apache.flink.streaming.api.function.source.SourceFunction;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class StreamVertexTest {
+
+	private static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
+
+	public static class MySource implements SourceFunction<Tuple1<Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private Tuple1<Integer> tuple = new Tuple1<Integer>(0);
+
+		@Override
+		public void invoke(Collector<Tuple1<Integer>> collector) throws Exception {
+			for (int i = 0; i < 10; i++) {
+				tuple.f0 = i;
+				collector.collect(tuple);
+			}
+		}
+	}
+
+	public static class MyTask extends RichMapFunction<Tuple1<Integer>, Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> map(Tuple1<Integer> value) throws Exception {
+			Integer i = value.f0;
+			return new Tuple2<Integer, Integer>(i, i + 1);
+		}
+	}
+
+	public static class MySink implements SinkFunction<Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void invoke(Tuple2<Integer, Integer> tuple) {
+			Integer k = tuple.getField(0);
+			Integer v = tuple.getField(1);
+			data.put(k, v);
+		}
+	}
+
+	@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);
+
+
+		try {
+			env.fromCollection(null);
+			fail();
+		} catch (NullPointerException e) {
+		}
+
+		try {
+			env.fromElements();
+			fail();
+		} catch (IllegalArgumentException e) {
+		}
+
+		try {
+			env.generateSequence(-10, -30);
+			fail();
+		} catch (IllegalArgumentException e) {
+		}
+
+		try {
+			env.setBufferTimeout(-10);
+			fail();
+		} catch (IllegalArgumentException e) {
+		}
+
+		try {
+			env.setExecutionParallelism(-10);
+			fail();
+		} catch (IllegalArgumentException e) {
+		}
+
+		try {
+			env.readTextFile("random/path/that/is/not/valid");
+			fail();
+		} catch (IllegalArgumentException e) {
+		}
+	}
+
+	private static class CoMap implements CoMapFunction<String, Long, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String map1(String value) {
+			return value;
+		}
+
+		@Override
+		public String map2(Long value) {
+			return value.toString();
+		}
+	}
+
+	static HashSet<String> resultSet;
+
+	private static class SetSink implements SinkFunction<String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void invoke(String value) {
+			resultSet.add(value);
+		}
+	}
+
+	@Test
+	public void coTest() throws Exception {
+		LocalStreamEnvironment env = StreamExecutionEnvironment
+				.createLocalEnvironment(SOURCE_PARALELISM);
+
+		DataStream<String> fromStringElements = env.fromElements("aa", "bb", "cc");
+		DataStream<Long> generatedSequence = env.generateSequence(0, 3);
+
+		fromStringElements.connect(generatedSequence).map(new CoMap()).addSink(new SetSink());
+
+		resultSet = new HashSet<String>();
+		env.execute();
+
+		HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
+				"2", "3"));
+		assertEquals(expectedSet, resultSet);
+	}
+
+	@Test
+	public void runStream() throws Exception {
+		LocalStreamEnvironment env = StreamExecutionEnvironment
+				.createLocalEnvironment(SOURCE_PARALELISM);
+
+		env.addSource(new MySource(), SOURCE_PARALELISM).map(new MyTask()).addSink(new MySink());
+
+		env.executeTest(MEMORYSIZE);
+		assertEquals(10, data.keySet().size());
+
+		for (Integer k : data.keySet()) {
+			assertEquals((Integer) (k + 1), data.get(k));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockInvokable.java
index 07314ea..c06f53a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockInvokable.java
@@ -25,7 +25,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
 import org.apache.flink.util.Collector;
@@ -88,7 +88,7 @@ public class MockInvokable<IN, OUT> {
 		return iterator;
 	}
 
-	public static <IN, OUT> List<OUT> createAndExecute(StreamOperatorInvokable<IN, OUT> invokable, List<IN> inputs) {
+	public static <IN, OUT> List<OUT> createAndExecute(StreamInvokable<IN, OUT> invokable, List<IN> inputs) {
 		MockInvokable<IN, OUT> mock = new MockInvokable<IN, OUT>(inputs);
 		invokable.initialize(mock.getCollector(), mock.getIterator(), mock.getInDeserializer(), false);
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
index 3baa08d..88673a3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockRecordWriterFactory.java
@@ -21,7 +21,7 @@ import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
 
 import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.streamcomponent.MockRecordWriter;
+import org.apache.flink.streaming.api.streamvertex.MockRecordWriter;
 import org.mockito.Mockito;
 
 public class MockRecordWriterFactory {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
index 4232398..211daf6 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/window/join/WindowJoinLocal.java
@@ -23,8 +23,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 public class WindowJoinLocal {
 
-	private static final int PARALLELISM = 1;
-	private static final int SOURCE_PARALLELISM = 1;
+	private static final int PARALLELISM = 4;
+	private static final int SOURCE_PARALLELISM = 2;
 
 	// This example will join two streams with a sliding window. One which emits
 	// people's grades and one which emits people's salaries.


[08/12] git commit: [FLINK-1103] [streaming] Updated WordCount example to become self-contained and removed old TestDataUtil

Posted by mb...@apache.org.
[FLINK-1103] [streaming] Updated WordCount example to become self-contained and removed old TestDataUtil


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

Branch: refs/heads/master
Commit: 30ac9fe650b833bea2a9ee61b7b2f34f6181eb6d
Parents: 2dc5437
Author: mbalassi <ba...@gmail.com>
Authored: Wed Sep 24 18:08:49 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/util/TestDataUtil.java      | 118 ---------------
 .../flink/streaming/util/TestDataUtilTest.java  |  44 ------
 .../flink-streaming-examples/pom.xml            |   6 +
 .../streaming/examples/wordcount/WordCount.java | 149 +++++++++++++++++++
 .../examples/wordcount/WordCountLocal.java      |  59 --------
 .../testdata_checksum/ASTopology.data.md5       |   1 -
 .../testdata_checksum/MovieLens100k.data.md5    |   1 -
 .../resources/testdata_checksum/hamlet.txt.md5  |   1 -
 .../testdata_checksum/terainput.txt.md5         |   1 -
 9 files changed, 155 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
deleted file mode 100644
index ad42f1f..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/TestDataUtil.java
+++ /dev/null
@@ -1,118 +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.util;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.MalformedURLException;
-import java.net.URL;
-
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestDataUtil {
-
-	// TODO: Exception handling
-	// TODO: check checksum after download
-	private static final Logger LOG = LoggerFactory.getLogger(TestDataUtil.class);
-	public static final String testDataDir = "src/test/resources/testdata/";
-	public static final String testRepoUrl = "http://info.ilab.sztaki.hu/~mbalassi/flink-streaming/testdata/";
-	public static final String testChekSumDir = "src/test/resources/testdata_checksum/";
-
-	public static void downloadIfNotExists(String fileName) {
-
-		File file = new File(testDataDir + fileName);
-		File checkFile = new File(testChekSumDir + fileName + ".md5");
-		String checkSumDesired = new String();
-		String checkSumActaul = new String();
-
-		File testDataDirectory = new File(testDataDir);
-		testDataDirectory.mkdirs();
-
-		try {
-			FileReader fileReader = new FileReader(checkFile);
-			BufferedReader bufferedReader = new BufferedReader(fileReader);
-			checkSumDesired = bufferedReader.readLine();
-			bufferedReader.close();
-			fileReader.close();
-		} catch (FileNotFoundException e) {
-			throw new RuntimeException("File not found: " + file.getAbsolutePath(), e);
-		} catch (IOException e) {
-			throw new RuntimeException("Cannot read file: " + file.getAbsolutePath(), e);
-		}
-
-		if (file.exists()) {
-			if (LOG.isInfoEnabled()) {
-				LOG.info("{} already exists.", fileName);
-			}
-
-			try {
-				checkSumActaul = DigestUtils.md5Hex(FileUtils.readFileToByteArray(file));
-			} catch (IOException e) {
-				throw new RuntimeException("Cannot read file to byte array: "
-						+ file.getAbsolutePath(), e);
-			}
-			if (!checkSumActaul.equals(checkSumDesired)) {
-				if (LOG.isInfoEnabled()) {
-					LOG.info("Checksum is incorrect.");
-					LOG.info("Downloading file.");
-				}
-				download(fileName);
-			}
-		} else {
-			if (LOG.isInfoEnabled()) {
-				LOG.info("File does not exist.");
-				LOG.info("Downloading file.");
-			}
-			download(fileName);
-		}
-
-	}
-
-	public static void download(String fileName) {
-		if (LOG.isInfoEnabled()) {
-			LOG.info("downloading {}", fileName);
-		}
-		
-		try {
-			URL website = new URL(testRepoUrl + fileName);
-			BufferedReader bReader = new BufferedReader(new InputStreamReader(website.openStream()));
-			File outFile = new File(testDataDir + fileName);
-			BufferedWriter bWriter = new BufferedWriter(new FileWriter(outFile));
-
-			String line;
-			while ((line = bReader.readLine()) != null) {
-				bWriter.write(line);
-				bWriter.newLine();
-			}
-			bWriter.close();
-		} catch (MalformedURLException e) {
-			throw new RuntimeException("URL is malformed: ", e);
-		} catch (IOException e) {
-			throw new RuntimeException("Unexpected problem while downloading file " + fileName, e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestDataUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestDataUtilTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestDataUtilTest.java
deleted file mode 100644
index fc11bcc..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestDataUtilTest.java
+++ /dev/null
@@ -1,44 +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.util;
-
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-
-public class TestDataUtilTest {
-
-	@SuppressWarnings("resource")
-	public boolean compareFile(String file1, String file2) throws FileNotFoundException,
-			IOException {
-
-		BufferedReader myInput1 = new BufferedReader(new InputStreamReader(new FileInputStream(
-				file1)));
-		BufferedReader myInput2 = new BufferedReader(new InputStreamReader(new FileInputStream(
-				file2)));
-
-		String line1, line2;
-		while ((line1 = myInput1.readLine()) != null && (line2 = myInput2.readLine()) != null) {
-			if (!line1.equals(line2))
-				return false;
-		}
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
index 9c9f00d..7222879 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
@@ -44,6 +44,12 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java-examples</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+        <dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-connectors</artifactId>
 			<version>${project.version}</version>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
new file mode 100644
index 0000000..3be0c89
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
@@ -0,0 +1,149 @@
+/*
+ * 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.examples.wordcount;
+
+import java.util.StringTokenizer;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+
+/**
+ * Implements the "WordCount" program that computes a simple word occurrence
+ * histogram over text files in a streaming fashion.
+ * 
+ * <p>
+ * The input is a plain text file with lines separated by newline characters.
+ * 
+ * <p>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from
+ * {@link WordCountData}.
+ * 
+ * <p>
+ * This example shows how to:
+ * <ul>
+ * <li>write a simple Flink Streaming program.
+ * <li>use Tuple data types.
+ * <li>write and use user-defined functions.
+ * </ul>
+ * 
+ */
+public class WordCount {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> text = getTextDataStream(env);
+
+		DataStream<Tuple2<String, Integer>> counts =
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				text.flatMap(new Tokenizer())
+				// group by the tuple field "0" and sum up tuple field "1"
+				.groupBy(0)
+				.sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath, 1);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute();
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a
+	 * user-defined FlatMapFunction. The function takes a line (String) and
+	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
+	 * Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String inTuple, Collector<Tuple2<String, Integer>> out)
+				throws Exception {
+			// tokenize the line
+			StringTokenizer tokenizer = new StringTokenizer(inTuple);
+
+			// emit the pairs
+			while (tokenizer.hasMoreTokens()) {
+				out.collect(new Tuple2<String, Integer>(tokenizer.nextToken(), 1));
+			}
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: WordCount <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WordCount example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: WordCount <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return env.fromElements(WordCountData.WORDS);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
deleted file mode 100644
index 9ffeeb1..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCountLocal.java
+++ /dev/null
@@ -1,59 +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.examples.wordcount;
-
-import java.util.StringTokenizer;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-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.util.TestDataUtil;
-import org.apache.flink.util.Collector;
-
-// This example will count the occurrence of each word in the input file.
-public class WordCountLocal {
-
-	public static class WordCountSplitter implements
-			FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String inTuple, Collector<Tuple2<String, Integer>> out)
-				throws Exception {
-			StringTokenizer tokenizer = new StringTokenizer(inTuple);
-			while (tokenizer.hasMoreTokens()) {
-				out.collect(new Tuple2<String, Integer>(tokenizer.nextToken(), 1));
-			}
-		}
-	}
-
-	public static void main(String[] args) throws Exception {
-
-		TestDataUtil.downloadIfNotExists("hamlet.txt");
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		DataStream<Tuple2<String, Integer>> dataStream = env
-				.readTextFile("src/test/resources/testdata/hamlet.txt")
-				.flatMap(new WordCountSplitter()).groupBy(0).sum(1);
-
-		dataStream.print();
-
-		env.execute();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
deleted file mode 100644
index 2c386b7..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
+++ /dev/null
@@ -1 +0,0 @@
-f1b947a26b33b32f1de2cdd841f7b4c8

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
deleted file mode 100644
index 6499b43..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
+++ /dev/null
@@ -1 +0,0 @@
-6e47046882bad158b0efbb84cd5cb987

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
deleted file mode 100644
index 6526a51..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
+++ /dev/null
@@ -1 +0,0 @@
-4bb8c10cdde12a4953250423266465cc

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/30ac9fe6/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5 b/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
deleted file mode 100644
index 365f210..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
+++ /dev/null
@@ -1 +0,0 @@
-7002e15fe547614160a0df6f22a5b8d0


[07/12] git commit: [streaming] Streaming jobgraph and vertex refactor to match recent runtime changes

Posted by mb...@apache.org.
[streaming] Streaming jobgraph and vertex refactor to match recent runtime changes


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

Branch: refs/heads/master
Commit: 73371101e9edd16f2823ce8af0b27283e3ed3264
Parents: 7cc2400
Author: Gyula Fora <gy...@apache.org>
Authored: Sun Sep 21 22:58:18 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    | 519 ++++++++-----------
 .../flink/streaming/api/StreamConfig.java       |  25 +-
 .../api/datastream/BatchedDataStream.java       |   4 +-
 .../streaming/api/datastream/DataStream.java    |  30 +-
 .../api/datastream/IterativeDataStream.java     |   2 +-
 .../environment/StreamExecutionEnvironment.java |  16 +-
 .../streaming/api/invokable/SinkInvokable.java  |   2 +-
 .../api/invokable/SourceInvokable.java          |  14 +-
 .../api/invokable/StreamInvokable.java          | 100 +++-
 .../api/invokable/StreamOperatorInvokable.java  | 119 -----
 .../operator/BatchGroupReduceInvokable.java     |   4 +-
 .../operator/BatchReduceInvokable.java          |   4 +-
 .../invokable/operator/CounterInvokable.java    |   4 +-
 .../api/invokable/operator/FilterInvokable.java |   4 +-
 .../invokable/operator/FlatMapInvokable.java    |   4 +-
 .../api/invokable/operator/MapInvokable.java    |   4 +-
 .../operator/StreamReduceInvokable.java         |   4 +-
 .../api/invokable/operator/co/CoInvokable.java  |  17 +-
 .../AbstractStreamComponent.java                |  71 ---
 .../api/streamcomponent/CoStreamTask.java       | 114 ----
 .../api/streamcomponent/InputHandler.java       | 104 ----
 .../api/streamcomponent/OutputHandler.java      | 179 -------
 .../StreamComponentException.java               |  68 ---
 .../streamcomponent/StreamIterationSink.java    | 104 ----
 .../streamcomponent/StreamIterationSource.java  | 100 ----
 .../api/streamcomponent/StreamSink.java         |  61 ---
 .../api/streamcomponent/StreamSource.java       |  52 --
 .../api/streamcomponent/StreamTask.java         |  55 --
 .../api/streamvertex/CoStreamVertex.java        | 113 ++++
 .../api/streamvertex/InputHandler.java          | 109 ++++
 .../api/streamvertex/OutputHandler.java         | 181 +++++++
 .../api/streamvertex/StreamIterationHead.java   | 100 ++++
 .../api/streamvertex/StreamIterationTail.java   | 104 ++++
 .../api/streamvertex/StreamVertex.java          |  96 ++++
 .../api/streamvertex/StreamVertexException.java |  68 +++
 .../api/collector/StreamCollectorTest.java      |   6 +-
 .../api/streamcomponent/MockRecordWriter.java   |  45 --
 .../streamcomponent/StreamComponentTest.java    | 197 -------
 .../api/streamvertex/MockRecordWriter.java      |  45 ++
 .../api/streamvertex/StreamVertexTest.java      | 184 +++++++
 .../flink/streaming/util/MockInvokable.java     |   4 +-
 .../streaming/util/MockRecordWriterFactory.java |   2 +-
 .../examples/window/join/WindowJoinLocal.java   |   4 +-
 43 files changed, 1399 insertions(+), 1643 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index d8d4f2d..a04dbaa 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -19,32 +19,29 @@ package org.apache.flink.streaming.api;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.streaming.api.collector.OutputSelector;
-import org.apache.flink.streaming.api.invokable.SinkInvokable;
-import org.apache.flink.streaming.api.invokable.SourceInvokable;
 import org.apache.flink.streaming.api.invokable.StreamInvokable;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.api.streamcomponent.CoStreamTask;
-import org.apache.flink.streaming.api.streamcomponent.StreamIterationSink;
-import org.apache.flink.streaming.api.streamcomponent.StreamIterationSource;
-import org.apache.flink.streaming.api.streamcomponent.StreamSink;
-import org.apache.flink.streaming.api.streamcomponent.StreamSource;
-import org.apache.flink.streaming.api.streamcomponent.StreamTask;
+import org.apache.flink.streaming.api.streamvertex.CoStreamVertex;
+import org.apache.flink.streaming.api.streamvertex.StreamIterationHead;
+import org.apache.flink.streaming.api.streamvertex.StreamIterationTail;
+import org.apache.flink.streaming.api.streamvertex.StreamVertex;
 import org.apache.flink.streaming.partitioner.ForwardPartitioner;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Object for building Apache Flink stream processing job graphs
@@ -55,8 +52,8 @@ public class JobGraphBuilder {
 	private final JobGraph jobGraph;
 
 	// Graph attributes
-	private Map<String, AbstractJobVertex> components;
-	private Map<String, Integer> componentParallelism;
+	private Map<String, AbstractJobVertex> streamVertices;
+	private Map<String, Integer> vertexParallelism;
 	private Map<String, Long> bufferTimeout;
 	private Map<String, List<String>> outEdgeList;
 	private Map<String, List<Integer>> outEdgeType;
@@ -66,17 +63,17 @@ public class JobGraphBuilder {
 	private Map<String, List<String>> inEdgeList;
 	private Map<String, List<StreamPartitioner<?>>> connectionTypes;
 	private Map<String, String> operatorNames;
-	private Map<String, StreamInvokable<?>> invokableObjects;
+	private Map<String, StreamInvokable<?, ?>> invokableObjects;
 	private Map<String, TypeSerializerWrapper<?>> typeWrapperIn1;
 	private Map<String, TypeSerializerWrapper<?>> typeWrapperIn2;
 	private Map<String, TypeSerializerWrapper<?>> typeWrapperOut1;
 	private Map<String, TypeSerializerWrapper<?>> typeWrapperOut2;
 	private Map<String, byte[]> serializedFunctions;
 	private Map<String, byte[]> outputSelectors;
-	private Map<String, Class<? extends AbstractInvokable>> componentClasses;
+	private Map<String, Class<? extends AbstractInvokable>> vertexClasses;
 	private Map<String, String> iterationIds;
-	private Map<String, String> iterationIDtoSourceName;
-	private Map<String, String> iterationIDtoSinkName;
+	private Map<String, String> iterationIDtoHeadName;
+	private Map<String, String> iterationIDtoTailName;
 	private Map<String, Integer> iterationTailCount;
 	private Map<String, Long> iterationWaitTime;
 
@@ -86,7 +83,6 @@ public class JobGraphBuilder {
 	/**
 	 * Creates an new {@link JobGraph} with the given name. A JobGraph is a DAG
 	 * and consists of sources, tasks (intermediate vertices) and sinks. A
-	 * JobGraph must contain at least a source and a sink.
 	 * 
 	 * @param jobGraphName
 	 *            Name of the JobGraph
@@ -95,8 +91,8 @@ public class JobGraphBuilder {
 
 		jobGraph = new JobGraph(jobGraphName);
 
-		components = new HashMap<String, AbstractJobVertex>();
-		componentParallelism = new HashMap<String, Integer>();
+		streamVertices = new HashMap<String, AbstractJobVertex>();
+		vertexParallelism = new HashMap<String, Integer>();
 		bufferTimeout = new HashMap<String, Long>();
 		outEdgeList = new HashMap<String, List<String>>();
 		outEdgeType = new HashMap<String, List<Integer>>();
@@ -106,17 +102,17 @@ public class JobGraphBuilder {
 		inEdgeList = new HashMap<String, List<String>>();
 		connectionTypes = new HashMap<String, List<StreamPartitioner<?>>>();
 		operatorNames = new HashMap<String, String>();
-		invokableObjects = new HashMap<String, StreamInvokable<?>>();
+		invokableObjects = new HashMap<String, StreamInvokable<?, ?>>();
 		typeWrapperIn1 = new HashMap<String, TypeSerializerWrapper<?>>();
 		typeWrapperIn2 = new HashMap<String, TypeSerializerWrapper<?>>();
 		typeWrapperOut1 = new HashMap<String, TypeSerializerWrapper<?>>();
 		typeWrapperOut2 = new HashMap<String, TypeSerializerWrapper<?>>();
 		serializedFunctions = new HashMap<String, byte[]>();
 		outputSelectors = new HashMap<String, byte[]>();
-		componentClasses = new HashMap<String, Class<? extends AbstractInvokable>>();
+		vertexClasses = new HashMap<String, Class<? extends AbstractInvokable>>();
 		iterationIds = new HashMap<String, String>();
-		iterationIDtoSourceName = new HashMap<String, String>();
-		iterationIDtoSinkName = new HashMap<String, String>();
+		iterationIDtoHeadName = new HashMap<String, String>();
+		iterationIDtoTailName = new HashMap<String, String>();
 		iterationTailCount = new HashMap<String, Integer>();
 		iterationWaitTime = new HashMap<String, Long>();
 
@@ -142,12 +138,16 @@ public class JobGraphBuilder {
 	}
 
 	/**
-	 * Adds source to the JobGraph with the given parameters
+	 * Adds a vertex to the streaming JobGraph with the given parameters
 	 * 
-	 * @param componentName
-	 *            Name of the component
-	 * @param InvokableObject
+	 * @param vertexName
+	 *            Name of the vertex
+	 * @param invokableObject
 	 *            User defined operator
+	 * @param inTypeWrapper
+	 *            Input type wrapper for serialization
+	 * @param outTypeWrapper
+	 *            Output type wrapper for serialization
 	 * @param operatorName
 	 *            Operator type
 	 * @param serializedFunction
@@ -155,25 +155,27 @@ public class JobGraphBuilder {
 	 * @param parallelism
 	 *            Number of parallel instances created
 	 */
-	public void addSource(String componentName, SourceInvokable<?> InvokableObject,
+	public <IN, OUT> void addStreamVertex(String vertexName,
+			StreamInvokable<IN, OUT> invokableObject, TypeSerializerWrapper<?> inTypeWrapper,
 			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
-		addComponent(componentName, StreamSource.class, InvokableObject, operatorName,
+		addVertex(vertexName, StreamVertex.class, invokableObject, operatorName,
 				serializedFunction, parallelism);
-		addTypeWrappers(componentName, null, null, outTypeWrapper, null);
+
+		addTypeWrappers(vertexName, inTypeWrapper, null, outTypeWrapper, null);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SOURCE: {}", componentName);
+			LOG.debug("Vertex: {}", vertexName);
 		}
 	}
 
 	/**
-	 * Adds a source to the iteration head to the {@link JobGraph}. The iterated
-	 * tuples will be fed from this component back to the graph.
+	 * Adds a vertex for the iteration head to the {@link JobGraph}. The
+	 * iterated values will be fed from this vertex back to the graph.
 	 * 
-	 * @param componentName
-	 *            Name of the component
+	 * @param vertexName
+	 *            Name of the vertex
 	 * @param iterationHead
 	 *            Id of the iteration head
 	 * @param iterationID
@@ -183,143 +185,82 @@ public class JobGraphBuilder {
 	 * @param waitTime
 	 *            Max wait time for next record
 	 */
-	public void addIterationSource(String componentName, String iterationHead, String iterationID,
+	public void addIterationHead(String vertexName, String iterationHead, String iterationID,
 			int parallelism, long waitTime) {
 
-		addComponent(componentName, StreamIterationSource.class, null, null, null, parallelism);
+		addVertex(vertexName, StreamIterationHead.class, null, null, null, parallelism);
 
-		iterationIds.put(componentName, iterationID);
-		iterationIDtoSourceName.put(iterationID, componentName);
+		iterationIds.put(vertexName, iterationID);
+		iterationIDtoHeadName.put(iterationID, vertexName);
 
-		setBytesFrom(iterationHead, componentName);
+		setBytesFrom(iterationHead, vertexName);
 
-		setEdge(componentName, iterationHead,
-				connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0), 0,
-				new ArrayList<String>(), false);
+		setEdge(vertexName, iterationHead, connectionTypes
+				.get(inEdgeList.get(iterationHead).get(0)).get(0), 0, new ArrayList<String>(),
+				false);
 
-		iterationWaitTime.put(iterationIDtoSourceName.get(iterationID), waitTime);
+		iterationWaitTime.put(iterationIDtoHeadName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("ITERATION SOURCE: {}", componentName);
+			LOG.debug("ITERATION SOURCE: {}", vertexName);
 		}
 	}
 
 	/**
-	 * Adds a task to the JobGraph with the given parameters
+	 * Adds a vertex for the iteration tail to the {@link JobGraph}. The values
+	 * intended to be iterated will be sent to this sink from the iteration
+	 * head.
 	 * 
-	 * @param componentName
-	 *            Name of the component
-	 * @param taskInvokableObject
-	 *            User defined operator
-	 * @param inTypeWrapper
-	 *            Input type wrapper for serialization
-	 * @param outTypeWrapper
-	 *            Output type wrapper for serialization
-	 * @param operatorName
-	 *            Operator type
-	 * @param serializedFunction
-	 *            Serialized udf
+	 * @param vertexName
+	 *            Name of the vertex
+	 * @param iterationTail
+	 *            Id of the iteration tail
+	 * @param iterationID
+	 *            ID of iteration for mulitple iterations
 	 * @param parallelism
 	 *            Number of parallel instances created
+	 * @param waitTime
+	 *            Max waiting time for next record
 	 */
-	public <IN, OUT> void addTask(String componentName,
-			StreamOperatorInvokable<IN, OUT> taskInvokableObject,
-			TypeSerializerWrapper<?> inTypeWrapper, TypeSerializerWrapper<?> outTypeWrapper,
-			String operatorName, byte[] serializedFunction, int parallelism) {
+	public void addIterationTail(String vertexName, String iterationTail, String iterationID,
+			int parallelism, long waitTime) {
 
-		addComponent(componentName, StreamTask.class, taskInvokableObject, operatorName,
-				serializedFunction, parallelism);
+		addVertex(vertexName, StreamIterationTail.class, null, null, null, parallelism);
+
+		iterationIds.put(vertexName, iterationID);
+		iterationIDtoTailName.put(iterationID, vertexName);
 
-		addTypeWrappers(componentName, inTypeWrapper, null, outTypeWrapper, null);
+		setBytesFrom(iterationTail, vertexName);
+		iterationWaitTime.put(iterationIDtoTailName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("TASK: {}", componentName);
+			LOG.debug("ITERATION SINK: {}", vertexName);
 		}
+
 	}
 
-	public <IN1, IN2, OUT> void addCoTask(String componentName,
+	public <IN1, IN2, OUT> void addCoTask(String vertexName,
 			CoInvokable<IN1, IN2, OUT> taskInvokableObject,
 			TypeSerializerWrapper<?> in1TypeWrapper, TypeSerializerWrapper<?> in2TypeWrapper,
 			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
-		addComponent(componentName, CoStreamTask.class, taskInvokableObject, operatorName,
-				serializedFunction, parallelism);
-
-		addTypeWrappers(componentName, in1TypeWrapper, in2TypeWrapper, outTypeWrapper, null);
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("CO-TASK: {}", componentName);
-		}
-	}
-
-	/**
-	 * Adds sink to the JobGraph with the given parameters
-	 * 
-	 * @param componentName
-	 *            Name of the component
-	 * @param InvokableObject
-	 *            User defined operator
-	 * @param operatorName
-	 *            Operator type
-	 * @param serializedFunction
-	 *            Serialized udf
-	 * @param parallelism
-	 *            Number of parallel instances created
-	 */
-	public void addSink(String componentName, SinkInvokable<?> InvokableObject,
-			TypeSerializerWrapper<?> inTypeWrapper, String operatorName, byte[] serializedFunction,
-			int parallelism) {
-
-		addComponent(componentName, StreamSink.class, InvokableObject, operatorName,
+		addVertex(vertexName, CoStreamVertex.class, taskInvokableObject, operatorName,
 				serializedFunction, parallelism);
-		addTypeWrappers(componentName, inTypeWrapper, null, null, null);
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK: {}", componentName);
-		}
-
-	}
-
-	/**
-	 * Adds a sink to an iteration tail to the {@link JobGraph}. The tuples
-	 * intended to be iterated will be sent to this sink from the iteration
-	 * head.
-	 * 
-	 * @param componentName
-	 *            Name of the component
-	 * @param iterationTail
-	 *            Id of the iteration tail
-	 * @param iterationID
-	 *            ID of iteration for mulitple iterations
-	 * @param parallelism
-	 *            Number of parallel instances created
-	 * @param waitTime
-	 *            Max waiting time for next record
-	 */
-	public void addIterationSink(String componentName, String iterationTail, String iterationID,
-			int parallelism, long waitTime) {
-
-		addComponent(componentName, StreamIterationSink.class, null, null, null, parallelism);
-
-		iterationIds.put(componentName, iterationID);
-		iterationIDtoSinkName.put(iterationID, componentName);
 
-		setBytesFrom(iterationTail, componentName);
-		iterationWaitTime.put(iterationIDtoSinkName.get(iterationID), waitTime);
+		addTypeWrappers(vertexName, in1TypeWrapper, in2TypeWrapper, outTypeWrapper, null);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("ITERATION SINK: {}", componentName);
+			LOG.debug("CO-TASK: {}", vertexName);
 		}
-
 	}
 
 	/**
-	 * Sets component parameters in the JobGraph
+	 * Sets vertex parameters in the JobGraph
 	 * 
-	 * @param componentName
-	 *            Name of the component
-	 * @param componentClass
+	 * @param vertexName
+	 *            Name of the vertex
+	 * @param vertexClass
 	 *            The class of the vertex
 	 * @param invokableObject
 	 *            The user defined invokable object
@@ -330,114 +271,153 @@ public class JobGraphBuilder {
 	 * @param parallelism
 	 *            Number of parallel instances created
 	 */
-	private void addComponent(String componentName,
-			Class<? extends AbstractInvokable> componentClass, StreamInvokable<?> invokableObject,
-			String operatorName, byte[] serializedFunction, int parallelism) {
-
-		componentClasses.put(componentName, componentClass);
-		setParallelism(componentName, parallelism);
-		mutability.put(componentName, false);
-		invokableObjects.put(componentName, invokableObject);
-		operatorNames.put(componentName, operatorName);
-		serializedFunctions.put(componentName, serializedFunction);
-		outEdgeList.put(componentName, new ArrayList<String>());
-		outEdgeType.put(componentName, new ArrayList<Integer>());
-		outEdgeNames.put(componentName, new ArrayList<List<String>>());
-		outEdgeSelectAll.put(componentName, new ArrayList<Boolean>());
-		inEdgeList.put(componentName, new ArrayList<String>());
-		connectionTypes.put(componentName, new ArrayList<StreamPartitioner<?>>());
-		iterationTailCount.put(componentName, 0);
+	private void addVertex(String vertexName, Class<? extends AbstractInvokable> vertexClass,
+			StreamInvokable<?, ?> invokableObject, String operatorName, byte[] serializedFunction,
+			int parallelism) {
+
+		vertexClasses.put(vertexName, vertexClass);
+		setParallelism(vertexName, parallelism);
+		mutability.put(vertexName, false);
+		invokableObjects.put(vertexName, invokableObject);
+		operatorNames.put(vertexName, operatorName);
+		serializedFunctions.put(vertexName, serializedFunction);
+		outEdgeList.put(vertexName, new ArrayList<String>());
+		outEdgeType.put(vertexName, new ArrayList<Integer>());
+		outEdgeNames.put(vertexName, new ArrayList<List<String>>());
+		outEdgeSelectAll.put(vertexName, new ArrayList<Boolean>());
+		inEdgeList.put(vertexName, new ArrayList<String>());
+		connectionTypes.put(vertexName, new ArrayList<StreamPartitioner<?>>());
+		iterationTailCount.put(vertexName, 0);
 	}
 
-	private void addTypeWrappers(String componentName, TypeSerializerWrapper<?> in1,
+	private void addTypeWrappers(String vertexName, TypeSerializerWrapper<?> in1,
 			TypeSerializerWrapper<?> in2, TypeSerializerWrapper<?> out1,
 			TypeSerializerWrapper<?> out2) {
-		typeWrapperIn1.put(componentName, in1);
-		typeWrapperIn2.put(componentName, in2);
-		typeWrapperOut1.put(componentName, out1);
-		typeWrapperOut2.put(componentName, out2);
+		typeWrapperIn1.put(vertexName, in1);
+		typeWrapperIn2.put(vertexName, in2);
+		typeWrapperOut1.put(vertexName, out1);
+		typeWrapperOut2.put(vertexName, out2);
 	}
 
 	/**
 	 * Creates an {@link AbstractJobVertex} in the {@link JobGraph} and sets its
 	 * config parameters using the ones set previously.
 	 * 
-	 * @param componentName
-	 *            Name of the component for which the vertex will be created.
+	 * @param vertexName
+	 *            Name for which the vertex will be created.
 	 */
-	private void createVertex(String componentName) {
+	private void createVertex(String vertexName) {
 
 		// Get vertex attributes
-		Class<? extends AbstractInvokable> componentClass = componentClasses.get(componentName);
-		StreamInvokable<?> invokableObject = invokableObjects.get(componentName);
-		String operatorName = operatorNames.get(componentName);
-		byte[] serializedFunction = serializedFunctions.get(componentName);
-		int parallelism = componentParallelism.get(componentName);
-		byte[] outputSelector = outputSelectors.get(componentName);
+		Class<? extends AbstractInvokable> vertexClass = vertexClasses.get(vertexName);
+		StreamInvokable<?, ?> invokableObject = invokableObjects.get(vertexName);
+		String operatorName = operatorNames.get(vertexName);
+		byte[] serializedFunction = serializedFunctions.get(vertexName);
+		int parallelism = vertexParallelism.get(vertexName);
+		byte[] outputSelector = outputSelectors.get(vertexName);
 
 		// Create vertex object
-		AbstractJobVertex component = new AbstractJobVertex(componentName);
+		AbstractJobVertex vertex = new AbstractJobVertex(vertexName);
 
-		this.jobGraph.addVertex(component);
+		this.jobGraph.addVertex(vertex);
 
-		component.setInvokableClass(componentClass);
-		component.setParallelism(parallelism);
+		vertex.setInvokableClass(vertexClass);
+		vertex.setParallelism(parallelism);
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Parallelism set: {} for {}", parallelism, componentName);
+			LOG.debug("Parallelism set: {} for {}", parallelism, vertexName);
 		}
 
-		StreamConfig config = new StreamConfig(component.getConfiguration());
+		StreamConfig config = new StreamConfig(vertex.getConfiguration());
 
-		config.setMutability(mutability.get(componentName));
-		config.setBufferTimeout(bufferTimeout.get(componentName));
+		config.setMutability(mutability.get(vertexName));
+		config.setBufferTimeout(bufferTimeout.get(vertexName));
 
-		config.setTypeWrapperIn1(typeWrapperIn1.get(componentName));
-		config.setTypeWrapperIn2(typeWrapperIn2.get(componentName));
-		config.setTypeWrapperOut1(typeWrapperOut1.get(componentName));
-		config.setTypeWrapperOut2(typeWrapperOut2.get(componentName));
+		config.setTypeWrapperIn1(typeWrapperIn1.get(vertexName));
+		config.setTypeWrapperIn2(typeWrapperIn2.get(vertexName));
+		config.setTypeWrapperOut1(typeWrapperOut1.get(vertexName));
+		config.setTypeWrapperOut2(typeWrapperOut2.get(vertexName));
 
 		// Set vertex config
 		config.setUserInvokable(invokableObject);
-		config.setComponentName(componentName);
+		config.setVertexName(vertexName);
 		config.setFunction(serializedFunction, operatorName);
 		config.setOutputSelector(outputSelector);
 
-		if (componentClass.equals(StreamIterationSource.class)
-				|| componentClass.equals(StreamIterationSink.class)) {
-			config.setIterationId(iterationIds.get(componentName));
-			config.setIterationWaitTime(iterationWaitTime.get(componentName));
+		if (vertexClass.equals(StreamIterationHead.class)
+				|| vertexClass.equals(StreamIterationTail.class)) {
+			config.setIterationId(iterationIds.get(vertexName));
+			config.setIterationWaitTime(iterationWaitTime.get(vertexName));
+		}
+
+		streamVertices.put(vertexName, vertex);
+	}
+
+	/**
+	 * Connects two vertices with the given names, partitioning and channel type
+	 * 
+	 * @param upStreamVertexName
+	 *            Name of the upstream vertex, that will emit the values
+	 * @param downStreamVertexName
+	 *            Name of the downstream vertex, that will receive the values
+	 * @param partitionerObject
+	 *            The partitioner
+	 */
+	private <T> void connect(String upStreamVertexName, String downStreamVertexName,
+			StreamPartitioner<T> partitionerObject) {
+
+		AbstractJobVertex upStreamVertex = streamVertices.get(upStreamVertexName);
+		AbstractJobVertex downStreamVertex = streamVertices.get(downStreamVertexName);
+
+		StreamConfig config = new StreamConfig(upStreamVertex.getConfiguration());
+
+		if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
+			downStreamVertex
+					.connectNewDataSetAsInput(upStreamVertex, DistributionPattern.POINTWISE);
+		} else {
+			downStreamVertex
+					.connectNewDataSetAsInput(upStreamVertex, DistributionPattern.BIPARTITE);
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("CONNECTED: {} - {} -> {}", partitionerObject.getClass().getSimpleName(),
+					upStreamVertexName, downStreamVertexName);
 		}
 
-		components.put(componentName, component);
+		int outputIndex = upStreamVertex.getNumberOfProducedIntermediateDataSets() - 1;
+
+		config.setOutputName(outputIndex, outEdgeNames.get(upStreamVertexName).get(outputIndex));
+		config.setSelectAll(outputIndex, outEdgeSelectAll.get(upStreamVertexName).get(outputIndex));
+		config.setPartitioner(outputIndex, partitionerObject);
+		config.setNumberOfOutputChannels(outputIndex, vertexParallelism.get(downStreamVertexName));
 	}
 
 	/**
-	 * Sets the number of parallel instances created for the given component.
+	 * Sets the number of parallel instances created for the given vertex.
 	 * 
-	 * @param componentName
-	 *            Name of the component
+	 * @param vertexName
+	 *            Name of the vertex
 	 * @param parallelism
 	 *            Number of parallel instances created
 	 */
-	public void setParallelism(String componentName, int parallelism) {
-		componentParallelism.put(componentName, parallelism);
+	public void setParallelism(String vertexName, int parallelism) {
+		vertexParallelism.put(vertexName, parallelism);
 	}
 
-	public void setMutability(String componentName, boolean isMutable) {
-		mutability.put(componentName, isMutable);
+	public void setMutability(String vertexName, boolean isMutable) {
+		mutability.put(vertexName, isMutable);
 	}
 
-	public void setBufferTimeout(String componentName, long bufferTimeout) {
-		this.bufferTimeout.put(componentName, bufferTimeout);
+	public void setBufferTimeout(String vertexName, long bufferTimeout) {
+		this.bufferTimeout.put(vertexName, bufferTimeout);
 	}
 
 	/**
 	 * Connects two vertices in the JobGraph using the selected partitioner
 	 * settings
 	 * 
-	 * @param upStreamComponentName
+	 * @param upStreamVertexName
 	 *            Name of the upstream(output) vertex
-	 * @param downStreamComponentName
+	 * @param downStreamVertexName
 	 *            Name of the downstream(input) vertex
 	 * @param partitionerObject
 	 *            Partitioner object
@@ -446,55 +426,15 @@ public class JobGraphBuilder {
 	 * @param outputNames
 	 *            User defined names of the out edge
 	 */
-	public void setEdge(String upStreamComponentName, String downStreamComponentName,
+	public void setEdge(String upStreamVertexName, String downStreamVertexName,
 			StreamPartitioner<?> partitionerObject, int typeNumber, List<String> outputNames,
 			boolean selectAll) {
-		outEdgeList.get(upStreamComponentName).add(downStreamComponentName);
-		outEdgeType.get(upStreamComponentName).add(typeNumber);
-		inEdgeList.get(downStreamComponentName).add(upStreamComponentName);
-		connectionTypes.get(upStreamComponentName).add(partitionerObject);
-		outEdgeNames.get(upStreamComponentName).add(outputNames);
-		outEdgeSelectAll.get(upStreamComponentName).add(selectAll);
-	}
-
-	/**
-	 * Connects to JobGraph components with the given names, partitioning and
-	 * channel type
-	 * 
-	 * @param upStreamComponentName
-	 *            Name of the upstream component, that will emit the tuples
-	 * @param downStreamComponentName
-	 *            Name of the downstream component, that will receive the tuples
-	 * @param partitionerObject
-	 *            The partitioner
-	 */
-	private <T> void connect(String upStreamComponentName, String downStreamComponentName,
-			StreamPartitioner<T> partitionerObject) {
-
-		AbstractJobVertex upStreamComponent = components.get(upStreamComponentName);
-		AbstractJobVertex downStreamComponent = components.get(downStreamComponentName);
-
-		StreamConfig config = new StreamConfig(upStreamComponent.getConfiguration());
-
-		if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
-			downStreamComponent.connectNewDataSetAsInput(upStreamComponent, DistributionPattern.POINTWISE);
-		} else {
-			downStreamComponent.connectNewDataSetAsInput(upStreamComponent, DistributionPattern.BIPARTITE);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("CONNECTED: {} - {} -> {}", partitionerObject.getClass().getSimpleName(),
-					upStreamComponentName, downStreamComponentName);
-		}
-		
-		int outputIndex = upStreamComponent.getNumberOfProducedIntermediateDataSets() - 1;
-
-		config.setOutputName(outputIndex, outEdgeNames.get(upStreamComponentName).get(outputIndex));
-		config.setSelectAll(outputIndex,
-				outEdgeSelectAll.get(upStreamComponentName).get(outputIndex));
-		config.setPartitioner(outputIndex, partitionerObject);
-		config.setNumberOfOutputChannels(outputIndex,
-				componentParallelism.get(downStreamComponentName));
+		outEdgeList.get(upStreamVertexName).add(downStreamVertexName);
+		outEdgeType.get(upStreamVertexName).add(typeNumber);
+		inEdgeList.get(downStreamVertexName).add(upStreamVertexName);
+		connectionTypes.get(upStreamVertexName).add(partitionerObject);
+		outEdgeNames.get(upStreamVertexName).add(outputNames);
+		outEdgeSelectAll.get(upStreamVertexName).add(selectAll);
 	}
 
 	/**
@@ -507,33 +447,31 @@ public class JobGraphBuilder {
 	 *            ID of the iteration tail
 	 */
 	public void setIterationSourceSettings(String iterationID, String iterationTail) {
-		setParallelism(iterationIDtoSourceName.get(iterationID),
-				componentParallelism.get(iterationTail));
-		setBufferTimeout(iterationIDtoSourceName.get(iterationID), bufferTimeout.get(iterationTail));
+		setParallelism(iterationIDtoHeadName.get(iterationID), vertexParallelism.get(iterationTail));
+		setBufferTimeout(iterationIDtoHeadName.get(iterationID), bufferTimeout.get(iterationTail));
 	}
 
 	/**
-	 * Sets a user defined {@link OutputSelector} for the given component. Used
-	 * for directed emits.
+	 * Sets a user defined {@link OutputSelector} for the given vertex. Used for
+	 * directed emits.
 	 * 
-	 * @param componentName
-	 *            Name of the component for which the output selector will be
-	 *            set
+	 * @param vertexName
+	 *            Name of the vertex for which the output selector will be set
 	 * @param serializedOutputSelector
 	 *            Byte array representing the serialized output selector.
 	 */
-	public <T> void setOutputSelector(String componentName, byte[] serializedOutputSelector) {
-		outputSelectors.put(componentName, serializedOutputSelector);
+	public <T> void setOutputSelector(String vertexName, byte[] serializedOutputSelector) {
+		outputSelectors.put(vertexName, serializedOutputSelector);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Outputselector set for {}", componentName);
+			LOG.debug("Outputselector set for {}", vertexName);
 		}
 
 	}
 
 	/**
-	 * Sets udf operator and TypeSerializerWrapper from one component to
-	 * another, used with some sinks.
+	 * Sets udf operator and TypeSerializerWrapper from one vertex to another,
+	 * used with some sinks.
 	 * 
 	 * @param from
 	 *            from
@@ -559,39 +497,32 @@ public class JobGraphBuilder {
 		return typeWrapperOut1.get(id).getTypeInfo();
 	}
 
-//  TODO: This should be adjusted to the sharing groups
-//	/**
-//	 * Sets instance sharing between the given components
-//	 * 
-//	 * @param component1
-//	 *            Share will be called on this component
-//	 * @param component2
-//	 *            Share will be called to this component
-//	 */
-//	public void setInstanceSharing(String component1, String component2) {
-//		AbstractJobVertex c1 = components.get(component1);
-//		AbstractJobVertex c2 = components.get(component2);
-//
-//		c1.setVertexToShareInstancesWith(c2);
-//	}
-
 	/**
-	 * Sets all components to share with the one with highest parallelism
+	 * Sets slot sharing for the vertices.
 	 */
-	private void setAutomaticInstanceSharing() {
+	private void setSlotSharing() {
 		SlotSharingGroup shareGroup = new SlotSharingGroup();
 
-		for (AbstractJobVertex vertex : components.values()) {
+		for (AbstractJobVertex vertex : streamVertices.values()) {
 			vertex.setSlotSharingGroup(shareGroup);
 		}
+
+		for (String iterID : new HashSet<String>(iterationIds.values())) {
+			CoLocationGroup ccg = new CoLocationGroup();
+			AbstractJobVertex tail = streamVertices.get(iterationIDtoTailName.get(iterID));
+			AbstractJobVertex head = streamVertices.get(iterationIDtoHeadName.get(iterID));
+
+			ccg.addVertex(head);
+			ccg.addVertex(tail);
+		}
 	}
 
 	/**
 	 * Writes number of inputs into each JobVertex's config
 	 */
 	private void setNumberOfJobInputs() {
-		for (AbstractJobVertex component : components.values()) {
-			(new StreamConfig(component.getConfiguration())).setNumberOfInputs(component
+		for (AbstractJobVertex vertex : streamVertices.values()) {
+			(new StreamConfig(vertex.getConfiguration())).setNumberOfInputs(vertex
 					.getNumberOfInputs());
 		}
 	}
@@ -601,43 +532,43 @@ public class JobGraphBuilder {
 	 * config
 	 */
 	private void setNumberOfJobOutputs() {
-		for (AbstractJobVertex component : components.values()) {
-			(new StreamConfig(component.getConfiguration())).setNumberOfOutputs(component
+		for (AbstractJobVertex vertex : streamVertices.values()) {
+			(new StreamConfig(vertex.getConfiguration())).setNumberOfOutputs(vertex
 					.getNumberOfProducedIntermediateDataSets());
 		}
 	}
 
 	/**
-	 * Builds the {@link JobGraph} from the components with the edges and
-	 * settings provided.
+	 * Builds the {@link JobGraph} from the vertices with the edges and settings
+	 * provided.
 	 */
 	private void buildGraph() {
 
-		for (String componentName : outEdgeList.keySet()) {
-			createVertex(componentName);
+		for (String vertexName : outEdgeList.keySet()) {
+			createVertex(vertexName);
 		}
 
-		for (String upStreamComponentName : outEdgeList.keySet()) {
+		for (String upStreamVertexName : outEdgeList.keySet()) {
 			int i = 0;
 
-			List<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
+			List<Integer> outEdgeTypeList = outEdgeType.get(upStreamVertexName);
 
-			for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
-				StreamConfig downStreamComponentConfig = new StreamConfig(components.get(
-						downStreamComponentName).getConfiguration());
+			for (String downStreamVertexName : outEdgeList.get(upStreamVertexName)) {
+				StreamConfig downStreamVertexConfig = new StreamConfig(streamVertices.get(
+						downStreamVertexName).getConfiguration());
 
-				int inputNumber = downStreamComponentConfig.getNumberOfInputs();
+				int inputNumber = downStreamVertexConfig.getNumberOfInputs();
 
-				downStreamComponentConfig.setInputType(inputNumber++, outEdgeTypeList.get(i));
-				downStreamComponentConfig.setNumberOfInputs(inputNumber);
+				downStreamVertexConfig.setInputType(inputNumber++, outEdgeTypeList.get(i));
+				downStreamVertexConfig.setNumberOfInputs(inputNumber);
 
-				connect(upStreamComponentName, downStreamComponentName,
-						connectionTypes.get(upStreamComponentName).get(i));
+				connect(upStreamVertexName, downStreamVertexName,
+						connectionTypes.get(upStreamVertexName).get(i));
 				i++;
 			}
 		}
 
-		setAutomaticInstanceSharing();
+		setSlotSharing();
 		setNumberOfJobInputs();
 		setNumberOfJobOutputs();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
index 2c53fb3..6fac391 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamConfig.java
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.collector.OutputSelector;
 import org.apache.flink.streaming.api.invokable.StreamInvokable;
-import org.apache.flink.streaming.api.streamcomponent.StreamComponentException;
+import org.apache.flink.streaming.api.streamvertex.StreamVertexException;
 import org.apache.flink.streaming.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
@@ -46,7 +46,7 @@ public class StreamConfig {
 	private static final String DIRECTED_EMIT = "directedEmit";
 	private static final String FUNCTION_NAME = "operatorName";
 	private static final String FUNCTION = "operator";
-	private static final String COMPONENT_NAME = "componentName";
+	private static final String VERTEX_NAME = "vertexName";
 	private static final String SERIALIZEDUDF = "serializedudf";
 	private static final String USER_FUNCTION = "userfunction";
 	private static final String BUFFER_TIMEOUT = "bufferTimeout";
@@ -125,8 +125,13 @@ public class StreamConfig {
 
 		TypeSerializerWrapper<T> typeWrapper = (TypeSerializerWrapper<T>) SerializationUtils
 				.deserialize(serializedWrapper);
+		if (typeWrapper != null) {
+			return typeWrapper.getTypeInfo();
+
+		} else {
+			return null;
+		}
 
-		return typeWrapper.getTypeInfo();
 	}
 
 	public void setMutability(boolean isMutable) {
@@ -145,7 +150,7 @@ public class StreamConfig {
 		return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
 	}
 
-	public void setUserInvokable(StreamInvokable<?> invokableObject) {
+	public void setUserInvokable(StreamInvokable<?,?> invokableObject) {
 		if (invokableObject != null) {
 			config.setClass(USER_FUNCTION, invokableObject.getClass());
 
@@ -162,16 +167,16 @@ public class StreamConfig {
 		try {
 			return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
 		} catch (Exception e) {
-			throw new StreamComponentException("Cannot instantiate user function", e);
+			throw new StreamVertexException("Cannot instantiate user function", e);
 		}
 	}
 
-	public void setComponentName(String componentName) {
-		config.setString(COMPONENT_NAME, componentName);
+	public void setVertexName(String vertexName) {
+		config.setString(VERTEX_NAME, vertexName);
 	}
 
-	public String getComponentName() {
-		return config.getString(COMPONENT_NAME, null);
+	public String getVertexName() {
+		return config.getString(VERTEX_NAME, null);
 	}
 
 	public void setFunction(byte[] serializedFunction, String functionName) {
@@ -212,7 +217,7 @@ public class StreamConfig {
 		try {
 			return deserializeObject(config.getBytes(OUTPUT_SELECTOR, null));
 		} catch (Exception e) {
-			throw new StreamComponentException("Cannot deserialize and instantiate OutputSelector",
+			throw new StreamVertexException("Cannot deserialize and instantiate OutputSelector",
 					e);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
index 2258d4a..51f1467 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
@@ -26,7 +26,7 @@ import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MaxAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.MinAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
@@ -200,7 +200,7 @@ public class BatchedDataStream<OUT> {
 	}
 
 	private SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
-		StreamOperatorInvokable<OUT, OUT> invokable = getReduceInvokable(aggregate);
+		StreamInvokable<OUT, OUT> invokable = getReduceInvokable(aggregate);
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = dataStream.addFunction("batchReduce",
 				aggregate, dataStream.outTypeWrapper, dataStream.outTypeWrapper, invokable);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index ace76aa..23bc80d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -47,7 +47,7 @@ import org.apache.flink.streaming.api.function.sink.WriteFormatAsText;
 import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByBatches;
 import org.apache.flink.streaming.api.function.sink.WriteSinkFunctionByMillis;
 import org.apache.flink.streaming.api.invokable.SinkInvokable;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.invokable.operator.CounterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
@@ -558,7 +558,7 @@ public class DataStream<OUT> {
 	public SingleOutputStreamOperator<OUT, ?> max() {
 		return max(0);
 	}
-	
+
 	/**
 	 * Applies an aggregation that gives the count of the data point.
 	 * 
@@ -568,15 +568,16 @@ public class DataStream<OUT> {
 		TypeSerializerWrapper<OUT> inTypeWrapper = outTypeWrapper;
 		TypeSerializerWrapper<Long> outTypeWrapper = new ObjectTypeWrapper<Long>(new Long(0));
 
-		return addFunction("counter", null, inTypeWrapper, outTypeWrapper, new CounterInvokable<OUT>());
+		return addFunction("counter", null, inTypeWrapper, outTypeWrapper,
+				new CounterInvokable<OUT>());
 	}
 
 	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
 
 		StreamReduceInvokable<OUT> invokable = new StreamReduceInvokable<OUT>(aggregate);
 
-		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("reduce", aggregate, outTypeWrapper,
-				outTypeWrapper, invokable);
+		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("reduce", aggregate,
+				outTypeWrapper, outTypeWrapper, invokable);
 
 		return returnStream;
 	}
@@ -759,7 +760,8 @@ public class DataStream<OUT> {
 	private DataStreamSink<OUT> writeAsText(DataStream<OUT> inputStream, String path,
 			WriteFormatAsText<OUT> format, int batchSize, OUT endTuple) {
 		DataStreamSink<OUT> returnStream = addSink(inputStream,
-				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), inputStream.outTypeWrapper);
+				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple),
+				inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}
@@ -909,7 +911,8 @@ public class DataStream<OUT> {
 	private DataStreamSink<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
 			WriteFormatAsCsv<OUT> format, int batchSize, OUT endTuple) {
 		DataStreamSink<OUT> returnStream = addSink(inputStream,
-				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), inputStream.outTypeWrapper);
+				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple),
+				inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}
@@ -944,7 +947,7 @@ public class DataStream<OUT> {
 
 		DataStream<R> returnStream = new DataStreamSource<R>(environment, "iterationSource", null);
 
-		jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
+		jobGraphBuilder.addIterationHead(returnStream.getId(), this.getId(), iterationID,
 				degreeOfParallelism, waitTime);
 
 		return this.copy();
@@ -966,15 +969,14 @@ public class DataStream<OUT> {
 	 */
 	protected <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
 			final Function function, TypeSerializerWrapper<OUT> inTypeWrapper,
-			TypeSerializerWrapper<R> outTypeWrapper,
-			StreamOperatorInvokable<OUT, R> functionInvokable) {
+			TypeSerializerWrapper<R> outTypeWrapper, StreamInvokable<OUT, R> functionInvokable) {
 		DataStream<OUT> inputStream = this.copy();
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
 				functionName, outTypeWrapper);
 
 		try {
-			jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, inTypeWrapper,
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), functionInvokable, inTypeWrapper,
 					outTypeWrapper, functionName,
 					SerializationUtils.serialize((Serializable) function), degreeOfParallelism);
 		} catch (SerializationException e) {
@@ -1049,13 +1051,13 @@ public class DataStream<OUT> {
 	}
 
 	private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream,
-			SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT> typeWrapper) {
+			SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT> inTypeWrapper) {
 		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink",
 				outTypeWrapper);
 
 		try {
-			jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),
-					typeWrapper, "sink", SerializationUtils.serialize(sinkFunction),
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),
+					inTypeWrapper, null, "sink", SerializationUtils.serialize(sinkFunction),
 					degreeOfParallelism);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize SinkFunction");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index e619f36..41616c9 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -82,7 +82,7 @@ public class IterativeDataStream<IN> extends
 	public <R> DataStream<IN> closeWith(DataStream<IN> iterationTail, String iterationName) {
 		DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink", null);
 
-		jobGraphBuilder.addIterationSink(returnStream.getId(), iterationTail.getId(),
+		jobGraphBuilder.addIterationTail(returnStream.getId(), iterationTail.getId(),
 				iterationID.toString(), iterationTail.getParallelism(), waitTime);
 
 		jobGraphBuilder.setIterationSourceSettings(iterationID.toString(), iterationTail.getId());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 00f9082..c3231d7 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -230,8 +230,8 @@ public abstract class StreamExecutionEnvironment {
 
 		try {
 			SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
-			jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
-					outTypeWrapper, "source", SerializationUtils.serialize(function), 1);
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SourceInvokable<OUT>(function),
+					null, outTypeWrapper, "source", SerializationUtils.serialize(function), 1);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize elements");
 		}
@@ -267,8 +267,8 @@ public abstract class StreamExecutionEnvironment {
 		try {
 			SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
 
-			jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(
-					new FromElementsFunction<OUT>(data)), new ObjectTypeWrapper<OUT>(data
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SourceInvokable<OUT>(
+					new FromElementsFunction<OUT>(data)), null, new ObjectTypeWrapper<OUT>(data
 					.iterator().next()), "source", SerializationUtils.serialize(function), 1);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize collection");
@@ -311,8 +311,9 @@ public abstract class StreamExecutionEnvironment {
 				outTypeWrapper);
 
 		try {
-			jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
-					outTypeWrapper, "source", SerializationUtils.serialize(function), parallelism);
+			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SourceInvokable<OUT>(function),
+					null, outTypeWrapper, "source", SerializationUtils.serialize(function),
+					parallelism);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize SourceFunction");
 		}
@@ -454,7 +455,8 @@ public abstract class StreamExecutionEnvironment {
 	 * <p>
 	 * The program execution will be logged and displayed with a generated
 	 * default name.
-	 * @throws Exception 
+	 * 
+	 * @throws Exception
 	 **/
 	public abstract void execute() throws Exception;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
index 4c7138c..ec33224 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SinkInvokable.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.invokable;
 
 import org.apache.flink.streaming.api.function.sink.SinkFunction;
 
-public class SinkInvokable<IN> extends StreamOperatorInvokable<IN, IN> {
+public class SinkInvokable<IN> extends StreamInvokable<IN, IN> {
 	private static final long serialVersionUID = 1L;
 
 	private SinkFunction<IN> sinkFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
index 30c86f9..8c9df46 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/SourceInvokable.java
@@ -21,7 +21,7 @@ import java.io.Serializable;
 
 import org.apache.flink.streaming.api.function.source.SourceFunction;
 
-public class SourceInvokable<OUT> extends StreamInvokable<OUT> implements Serializable {
+public class SourceInvokable<OUT> extends StreamInvokable<OUT,OUT> implements Serializable {
 
 	private static final long serialVersionUID = 1L;
 
@@ -38,4 +38,16 @@ public class SourceInvokable<OUT> extends StreamInvokable<OUT> implements Serial
 		sourceFunction.invoke(collector);
 	}
 
+	@Override
+	protected void immutableInvoke() throws Exception {		
+	}
+
+	@Override
+	protected void mutableInvoke() throws Exception {		
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
index 4f0e138..b3cd57f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
@@ -22,18 +22,30 @@ import java.io.Serializable;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.RichFunction;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
 import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+import org.apache.flink.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * The StreamInvokable represents the base class for all invokables in
- * the streaming topology.
+ * The StreamInvokable represents the base class for all invokables in the
+ * streaming topology.
  *
  * @param <OUT>
  *            The output type of the invokable
  */
-public abstract class StreamInvokable<OUT> implements Serializable {
+public abstract class StreamInvokable<IN, OUT> implements Serializable {
 
 	private static final long serialVersionUID = 1L;
+	private static final Logger LOG = LoggerFactory.getLogger(StreamInvokable.class);
+
+	protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
+	protected StreamRecordSerializer<IN> serializer;
+	protected StreamRecord<IN> reuse;
+	protected boolean isMutable;
 
 	protected Collector<OUT> collector;
 	protected Function userFunction;
@@ -43,8 +55,79 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 		this.userFunction = userFunction;
 	}
 
-	public void setCollector(Collector<OUT> collector) {
+	/**
+	 * Initializes the {@link StreamOperatorInvokable} for input and output
+	 * handling
+	 * 
+	 * @param collector
+	 *            Collector object for collecting the outputs for the operator
+	 * @param recordIterator
+	 *            Iterator for reading in the input records
+	 * @param serializer
+	 *            Serializer used to deserialize inputs
+	 * @param isMutable
+	 *            Mutability setting for the operator
+	 */
+	public void initialize(Collector<OUT> collector,
+			MutableObjectIterator<StreamRecord<IN>> recordIterator,
+			StreamRecordSerializer<IN> serializer, boolean isMutable) {
 		this.collector = collector;
+		this.recordIterator = recordIterator;
+		this.serializer = serializer;
+		if(this.serializer != null){
+			this.reuse = serializer.createInstance();
+		}
+		this.isMutable = isMutable;
+	}
+
+	/**
+	 * Re-initializes the object in which the next input record will be read in
+	 */
+	protected void resetReuse() {
+		this.reuse = serializer.createInstance();
+	}
+
+	/**
+	 * Method that will be called if the mutability setting is set to immutable
+	 */
+	protected abstract void immutableInvoke() throws Exception;
+
+	/**
+	 * Method that will be called if the mutability setting is set to mutable
+	 */
+	protected abstract void mutableInvoke() throws Exception;
+
+	/**
+	 * The call of the user implemented function should be implemented here
+	 */
+	protected abstract void callUserFunction() throws Exception;
+
+	/**
+	 * Method for logging exceptions thrown during the user function call
+	 */
+	protected void callUserFunctionAndLogException() {
+		try {
+			callUserFunction();
+		} catch (Exception e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Calling user function failed due to: {}",
+						StringUtils.stringifyException(e));
+			}
+		}
+	}
+
+	/**
+	 * Method that will be called when the stream starts. The user should encode
+	 * the processing functionality in {@link #mutableInvoke()} and
+	 * {@link #immutableInvoke()}
+	 * 
+	 */
+	public void invoke() throws Exception {
+		if (this.isMutable) {
+			mutableInvoke();
+		} else {
+			immutableInvoke();
+		}
 	}
 
 	/**
@@ -55,7 +138,7 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 	 *            The configuration parameters for the operator
 	 */
 	public void open(Configuration parameters) throws Exception {
-		isRunning=true;
+		isRunning = true;
 		if (userFunction instanceof RichFunction) {
 			((RichFunction) userFunction).open(parameters);
 		}
@@ -72,11 +155,4 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 			((RichFunction) userFunction).close();
 		}
 	}
-
-	/**
-	 * The method that will be called once when the operator is created, the
-	 * working mechanics of the operator should be implemented here
-	 * 
-	 */
-	public abstract void invoke() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamOperatorInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamOperatorInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamOperatorInvokable.java
deleted file mode 100644
index 239f9f4..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamOperatorInvokable.java
+++ /dev/null
@@ -1,119 +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.invokable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-import org.apache.flink.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The StreamOperatorInvokable represents the base class for all operators in
- * the streaming topology.
- *
- * @param <IN>
- *            Input type of the operator
- * @param <OUT>
- *            Output type of the operator
- */
-public abstract class StreamOperatorInvokable<IN, OUT> extends StreamInvokable<OUT> {
-
-	public StreamOperatorInvokable(Function userFunction) {
-		super(userFunction);
-	}
-
-	private static final long serialVersionUID = 1L;
-	private static final Logger LOG = LoggerFactory.getLogger(StreamOperatorInvokable.class);
-
-	protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
-	protected StreamRecordSerializer<IN> serializer;
-	protected StreamRecord<IN> reuse;
-	protected boolean isMutable;
-
-	/**
-	 * Initializes the {@link StreamOperatorInvokable} for input and output
-	 * handling
-	 * 
-	 * @param collector
-	 *            Collector object for collecting the outputs for the operator
-	 * @param recordIterator
-	 *            Iterator for reading in the input records
-	 * @param serializer
-	 *            Serializer used to deserialize inputs
-	 * @param isMutable
-	 *            Mutability setting for the operator
-	 */
-	public void initialize(Collector<OUT> collector,
-			MutableObjectIterator<StreamRecord<IN>> recordIterator,
-			StreamRecordSerializer<IN> serializer, boolean isMutable) {
-		setCollector(collector);
-		this.recordIterator = recordIterator;
-		this.serializer = serializer;
-		this.reuse = serializer.createInstance();
-		this.isMutable = isMutable;
-	}
-
-	/**
-	 * Re-initializes the object in which the next input record will be read in
-	 */
-	protected void resetReuse() {
-		this.reuse = serializer.createInstance();
-	}
-
-	/**
-	 * Method that will be called if the mutability setting is set to immutable
-	 */
-	protected abstract void immutableInvoke() throws Exception;
-
-	/**
-	 * Method that will be called if the mutability setting is set to mutable
-	 */
-	protected abstract void mutableInvoke() throws Exception;
-
-	/**
-	 * The call of the user implemented function should be implemented here
-	 */
-	protected abstract void callUserFunction() throws Exception;
-
-	/**
-	 * Method for logging exceptions thrown during the user function call
-	 */
-	protected void callUserFunctionAndLogException() {
-		try {
-			callUserFunction();
-		} catch (Exception e) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Calling user function failed due to: {}",
-						StringUtils.stringifyException(e));
-			}
-		}
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		if (this.isMutable) {
-			mutableInvoke();
-		} else {
-			immutableInvoke();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
index 2e64d7c..81498dc 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceInvokable.java
@@ -24,11 +24,11 @@ import java.util.Iterator;
 import org.apache.commons.math.util.MathUtils;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 import org.apache.flink.streaming.state.SlidingWindowState;
 
-public class BatchGroupReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, OUT> {
+public class BatchGroupReduceInvokable<IN, OUT> extends StreamInvokable<IN, OUT> {
 
 	private static final long serialVersionUID = 1L;
 	protected GroupReduceFunction<IN, OUT> reducer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index 41ae3f3..f306dac 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -24,11 +24,11 @@ import org.apache.commons.math.util.MathUtils;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 import org.apache.flink.streaming.state.NullableCircularBuffer;
 
-public class BatchReduceInvokable<OUT> extends StreamOperatorInvokable<OUT, OUT> {
+public class BatchReduceInvokable<OUT> extends StreamInvokable<OUT, OUT> {
 
 	private static final long serialVersionUID = 1L;
 	protected ReduceFunction<OUT> reducer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java
index 12f43e1..7924595 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java
@@ -17,9 +17,9 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 
-public class CounterInvokable<IN> extends StreamOperatorInvokable<IN, Long> {
+public class CounterInvokable<IN> extends StreamInvokable<IN, Long> {
 	private static final long serialVersionUID = 1L;
 	
 	Long count = 0L;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
index 9e5b18a..a54b6ad 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FilterInvokable.java
@@ -18,9 +18,9 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 
-public class FilterInvokable<IN> extends StreamOperatorInvokable<IN, IN> {
+public class FilterInvokable<IN> extends StreamInvokable<IN, IN> {
 
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
index 1a0c93e..3452a82 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/FlatMapInvokable.java
@@ -18,9 +18,9 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 
-public class FlatMapInvokable<IN, OUT> extends StreamOperatorInvokable<IN, OUT> {
+public class FlatMapInvokable<IN, OUT> extends StreamInvokable<IN, OUT> {
 	private static final long serialVersionUID = 1L;
 
 	private FlatMapFunction<IN, OUT> flatMapper;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
index 40142ed..4feb4f3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/MapInvokable.java
@@ -18,9 +18,9 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 
-public class MapInvokable<IN, OUT> extends StreamOperatorInvokable<IN, OUT> {
+public class MapInvokable<IN, OUT> extends StreamInvokable<IN, OUT> {
 	private static final long serialVersionUID = 1L;
 
 	private MapFunction<IN, OUT> mapper;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
index 7080a7f..d327c76 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/StreamReduceInvokable.java
@@ -18,9 +18,9 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.api.invokable.StreamOperatorInvokable;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
 
-public class StreamReduceInvokable<IN> extends StreamOperatorInvokable<IN, IN> {
+public class StreamReduceInvokable<IN> extends StreamInvokable<IN, IN> {
 	private static final long serialVersionUID = 1L;
 
 	protected ReduceFunction<IN> reducer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
index 29e7650..098bbc6 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
+public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<IN1, OUT> {
 
 	public CoInvokable(Function userFunction) {
 		super(userFunction);
@@ -41,7 +41,6 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 	protected StreamRecord<IN2> reuse2;
 	protected StreamRecordSerializer<IN1> serializer1;
 	protected StreamRecordSerializer<IN2> serializer2;
-	protected boolean isMutable;
 
 	public void initialize(Collector<OUT> collector,
 			CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> recordIterator,
@@ -71,14 +70,7 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 		this.reuse2 = serializer2.createInstance();
 	}
 
-	public void invoke() throws Exception {
-		if (this.isMutable) {
-			mutableInvoke();
-		} else {
-			immutableInvoke();
-		}
-	}
-
+	@Override
 	protected void immutableInvoke() throws Exception {
 		while (true) {
 			int next = recordIterator.next(reuse1, reuse2);
@@ -96,6 +88,7 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 		}
 	}
 
+	@Override
 	protected void mutableInvoke() throws Exception {
 		while (true) {
 			int next = recordIterator.next(reuse1, reuse2);
@@ -149,4 +142,8 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 		}
 	}
 
+	@Override
+	protected void callUserFunction() throws Exception {
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.java
deleted file mode 100644
index 03c3988..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/AbstractStreamComponent.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.api.streamcomponent;
-
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.streaming.api.StreamConfig;
-import org.apache.flink.streaming.api.invokable.StreamInvokable;
-
-public abstract class AbstractStreamComponent extends AbstractInvokable {
-
-	protected StreamConfig configuration;
-	protected int instanceID;
-	protected String name;
-	private static int numComponents = 0;
-	protected boolean isMutable;
-	protected Object function;
-	protected String functionName;
-
-	protected static int newComponent() {
-		numComponents++;
-		return numComponents;
-	}
-
-	@Override
-	public void registerInputOutput() {
-		initialize();
-		setInputsOutputs();
-		setInvokable();
-	}
-	
-	protected void initialize() {
-		this.configuration = new StreamConfig(getTaskConfiguration());
-		this.name = configuration.getComponentName();
-		this.isMutable = configuration.getMutability();
-		this.functionName = configuration.getFunctionName();
-		this.function = configuration.getFunction();
-	}
-
-	protected <T> void invokeUserFunction(StreamInvokable<T> userInvokable) throws Exception {
-		userInvokable.open(getTaskConfiguration());
-		userInvokable.invoke();
-		userInvokable.close();
-	}
-
-	protected abstract void setInputsOutputs();
-
-	protected abstract void setInvokable();
-
-	public String getName() {
-		return name;
-	}
-
-	public int getInstanceID() {
-		return instanceID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73371101/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
deleted file mode 100644
index 4483969..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamcomponent/CoStreamTask.java
+++ /dev/null
@@ -1,114 +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.streamcomponent;
-
-import java.util.ArrayList;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.io.network.api.MutableRecordReader;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
-import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
-import org.apache.flink.streaming.io.CoReaderIterator;
-import org.apache.flink.streaming.io.CoRecordReader;
-import org.apache.flink.util.MutableObjectIterator;
-
-public class CoStreamTask<IN1 extends Tuple, IN2 extends Tuple, OUT extends Tuple> extends
-		AbstractStreamComponent {
-
-	private OutputHandler<OUT> outputHandler;
-
-	protected StreamRecordSerializer<IN1> inputDeserializer1 = null;
-	protected StreamRecordSerializer<IN2> inputDeserializer2 = null;
-
-	MutableObjectIterator<StreamRecord<IN1>> inputIter1;
-	MutableObjectIterator<StreamRecord<IN2>> inputIter2;
-
-	CoRecordReader<DeserializationDelegate<StreamRecord<IN1>>, DeserializationDelegate<StreamRecord<IN2>>> coReader;
-	CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> coIter;
-
-	private CoInvokable<IN1, IN2, OUT> userInvokable;
-	private static int numTasks;
-
-	public CoStreamTask() {
-		userInvokable = null;
-		numTasks = newComponent();
-		instanceID = numTasks;
-	}
-
-	private void setDeserializers() {
-		TypeInformation<IN1> inputTypeInfo1 = configuration.getTypeInfoIn1();
-		inputDeserializer1 = new StreamRecordSerializer<IN1>(inputTypeInfo1);
-
-		TypeInformation<IN2> inputTypeInfo2 = configuration.getTypeInfoIn2();
-		inputDeserializer2 = new StreamRecordSerializer<IN2>(inputTypeInfo2);
-	}
-
-	@Override
-	public void setInputsOutputs() {
-		outputHandler = new OutputHandler<OUT>(this);
-
-		setConfigInputs();
-
-		coIter = new CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>>(coReader,
-				inputDeserializer1, inputDeserializer2);
-	}
-
-	@Override
-	protected void setInvokable() {
-		userInvokable = configuration.getUserInvokable();
-		userInvokable.initialize(outputHandler.getCollector(), coIter, inputDeserializer1,
-				inputDeserializer2, isMutable);
-	}
-
-	protected void setConfigInputs() throws StreamComponentException {
-		setDeserializers();
-
-		int numberOfInputs = configuration.getNumberOfInputs();
-
-		ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>> inputList1 = new ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>>();
-		ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>> inputList2 = new ArrayList<MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>>();
-
-		for (int i = 0; i < numberOfInputs; i++) {
-			int inputType = configuration.getInputType(i);
-			switch (inputType) {
-			case 1:
-				inputList1.add(new MutableRecordReader<DeserializationDelegate<StreamRecord<IN1>>>(
-						this));
-				break;
-			case 2:
-				inputList2.add(new MutableRecordReader<DeserializationDelegate<StreamRecord<IN2>>>(
-						this));
-				break;
-			default:
-				throw new RuntimeException("Invalid input type number: " + inputType);
-			}
-		}
-
-		coReader = new CoRecordReader<DeserializationDelegate<StreamRecord<IN1>>, DeserializationDelegate<StreamRecord<IN2>>>(
-				inputList1, inputList2);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		outputHandler.invokeUserFunction("CO-TASK", userInvokable);
-	}
-
-}


[11/12] git commit: [FLINK-1122] [streaming] Job Execution with user specified name

Posted by mb...@apache.org.
[FLINK-1122] [streaming] Job Execution with user specified name


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

Branch: refs/heads/master
Commit: 076223cb9540c973eaba7d24e50c0e1f3eb80308
Parents: 70464bb
Author: mbalassi <ba...@gmail.com>
Authored: Wed Sep 24 21:05:58 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 21:07:13 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    | 53 +++++++++++---------
 .../api/environment/LocalStreamEnvironment.java | 18 ++++++-
 .../environment/RemoteStreamEnvironment.java    | 20 +++++++-
 .../environment/StreamExecutionEnvironment.java | 30 ++++++++---
 .../streaming/examples/wordcount/WordCount.java |  2 +-
 5 files changed, 86 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/076223cb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index 3377ee0..e06fde3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -49,7 +49,8 @@ import org.slf4j.LoggerFactory;
 public class JobGraphBuilder {
 
 	private static final Logger LOG = LoggerFactory.getLogger(JobGraphBuilder.class);
-	private final JobGraph jobGraph;
+	private final static String DEAFULT_JOB_NAME = "Streaming Job";
+	private JobGraph jobGraph;
 
 	// Graph attributes
 	private Map<String, AbstractJobVertex> streamVertices;
@@ -87,9 +88,7 @@ public class JobGraphBuilder {
 	 * @param jobGraphName
 	 *            Name of the JobGraph
 	 */
-	public JobGraphBuilder(String jobGraphName) {
-
-		jobGraph = new JobGraph(jobGraphName);
+	public JobGraphBuilder() {
 
 		streamVertices = new HashMap<String, AbstractJobVertex>();
 		vertexParallelism = new HashMap<String, Integer>();
@@ -157,8 +156,8 @@ public class JobGraphBuilder {
 	 */
 	public <IN, OUT> void addStreamVertex(String vertexName,
 			StreamInvokable<IN, OUT> invokableObject, TypeWrapper<?> inTypeWrapper,
-			TypeWrapper<?> outTypeWrapper, String operatorName,
-			byte[] serializedFunction, int parallelism) {
+			TypeWrapper<?> outTypeWrapper, String operatorName, byte[] serializedFunction,
+			int parallelism) {
 
 		addVertex(vertexName, StreamVertex.class, invokableObject, operatorName,
 				serializedFunction, parallelism);
@@ -240,9 +239,8 @@ public class JobGraphBuilder {
 	}
 
 	public <IN1, IN2, OUT> void addCoTask(String vertexName,
-			CoInvokable<IN1, IN2, OUT> taskInvokableObject,
-			TypeWrapper<?> in1TypeWrapper, TypeWrapper<?> in2TypeWrapper,
-			TypeWrapper<?> outTypeWrapper, String operatorName,
+			CoInvokable<IN1, IN2, OUT> taskInvokableObject, TypeWrapper<?> in1TypeWrapper,
+			TypeWrapper<?> in2TypeWrapper, TypeWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
 		addVertex(vertexName, CoStreamVertex.class, taskInvokableObject, operatorName,
@@ -290,9 +288,8 @@ public class JobGraphBuilder {
 		iterationTailCount.put(vertexName, 0);
 	}
 
-	private void addTypeWrappers(String vertexName, TypeWrapper<?> in1,
-			TypeWrapper<?> in2, TypeWrapper<?> out1,
-			TypeWrapper<?> out2) {
+	private void addTypeWrappers(String vertexName, TypeWrapper<?> in1, TypeWrapper<?> in2,
+			TypeWrapper<?> out1, TypeWrapper<?> out2) {
 		typeWrapperIn1.put(vertexName, in1);
 		typeWrapperIn2.put(vertexName, in2);
 		typeWrapperOut1.put(vertexName, out1);
@@ -539,11 +536,29 @@ public class JobGraphBuilder {
 	}
 
 	/**
+	 * Gets the assembled {@link JobGraph} and adds a default name for it.
+	 */
+	public JobGraph getJobGraph() {
+		return getJobGraph(DEAFULT_JOB_NAME);
+	}
+
+	/**
+	 * Gets the assembled {@link JobGraph} and adds a user specified name for
+	 * it.
+	 * 
+	 * @param jobGraphName name of the jobGraph
+	 */
+	public JobGraph getJobGraph(String jobGraphName) {
+		jobGraph = new JobGraph(jobGraphName);
+		buildJobGraph();
+		return jobGraph;
+	}
+
+	/**
 	 * Builds the {@link JobGraph} from the vertices with the edges and settings
 	 * provided.
 	 */
-	private void buildGraph() {
-
+	private void buildJobGraph() {
 		for (String vertexName : outEdgeList.keySet()) {
 			createVertex(vertexName);
 		}
@@ -573,14 +588,4 @@ public class JobGraphBuilder {
 		setNumberOfJobOutputs();
 	}
 
-	/**
-	 * Builds and returns the JobGraph
-	 * 
-	 * @return JobGraph object
-	 */
-	public JobGraph getJobGraph() {
-		buildGraph();
-		return jobGraph;
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/076223cb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 4f259d4..94e0891 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -22,16 +22,30 @@ import org.apache.flink.streaming.util.ClusterUtil;
 public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 
 	/**
-	 * Executes the JobGraph of the on a mini cluster of CLusterUtil.
-	 * 
+	 * Executes the JobGraph of the on a mini cluster of CLusterUtil with a
+	 * default name.
 	 */
 	@Override
 	public void execute() throws Exception {
 		ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism());
 	}
 
+	/**
+	 * Executes the JobGraph of the on a mini cluster of CLusterUtil with a user
+	 * specified name.
+	 * 
+	 * @param jobName
+	 *            name of the job
+	 */
+	@Override
+	public void execute(String jobName) throws Exception {
+		ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(jobName),
+				getExecutionParallelism());
+	}
+
 	public void executeTest(long memorySize) throws Exception {
 		ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism(),
 				memorySize);
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/076223cb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 0582668..864e18d 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -70,12 +70,28 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 
 	@Override
 	public void execute() {
+		
+		JobGraph jobGraph = jobGraphBuilder.getJobGraph();
+		executeRemotely(jobGraph);
+	}
+	
+	@Override
+	public void execute(String jobName) {
+		
+		JobGraph jobGraph = jobGraphBuilder.getJobGraph(jobName);
+		executeRemotely(jobGraph);
+	}
+
+	/**
+	 * Executes the remote job.
+	 * 
+	 * @param jobGraph jobGraph to execute
+	 */
+	private void executeRemotely(JobGraph jobGraph) {
 		if (LOG.isInfoEnabled()) {
 			LOG.info("Running remotely at {}:{}", host, port);
 		}
 
-		JobGraph jobGraph = jobGraphBuilder.getJobGraph();
-
 		for (int i = 0; i < jarFiles.length; i++) {
 			File file = new File(jarFiles[i]);
 			try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/076223cb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index e7a68d3..4d34217 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -71,7 +71,7 @@ public abstract class StreamExecutionEnvironment {
 	 * Constructor for creating StreamExecutionEnvironment
 	 */
 	protected StreamExecutionEnvironment() {
-		jobGraphBuilder = new JobGraphBuilder("jobGraph");
+		jobGraphBuilder = new JobGraphBuilder();
 	}
 
 	public int getExecutionParallelism() {
@@ -230,8 +230,9 @@ public abstract class StreamExecutionEnvironment {
 
 		try {
 			SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
-			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SourceInvokable<OUT>(function),
-					null, outTypeWrapper, "source", SerializationUtils.serialize(function), 1);
+			jobGraphBuilder.addStreamVertex(returnStream.getId(),
+					new SourceInvokable<OUT>(function), null, outTypeWrapper, "source",
+					SerializationUtils.serialize(function), 1);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize elements");
 		}
@@ -259,8 +260,7 @@ public abstract class StreamExecutionEnvironment {
 			throw new IllegalArgumentException("Collection must not be empty");
 		}
 
-		TypeWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data.iterator()
-				.next());
+		TypeWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data.iterator().next());
 		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements",
 				outTypeWrapper);
 
@@ -311,9 +311,9 @@ public abstract class StreamExecutionEnvironment {
 				outTypeWrapper);
 
 		try {
-			jobGraphBuilder.addStreamVertex(returnStream.getId(), new SourceInvokable<OUT>(function),
-					null, outTypeWrapper, "source", SerializationUtils.serialize(function),
-					parallelism);
+			jobGraphBuilder.addStreamVertex(returnStream.getId(),
+					new SourceInvokable<OUT>(function), null, outTypeWrapper, "source",
+					SerializationUtils.serialize(function), parallelism);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize SourceFunction");
 		}
@@ -461,6 +461,20 @@ public abstract class StreamExecutionEnvironment {
 	public abstract void execute() throws Exception;
 
 	/**
+	 * Triggers the program execution. The environment will execute all parts of
+	 * the program that have resulted in a "sink" operation. Sink operations are
+	 * for example printing results or forwarding them to a message queue.
+	 * <p>
+	 * The program execution will be logged and displayed with the provided
+	 * name
+	 * 
+	 * @param jobName Desired name of the job
+	 * 
+	 * @throws Exception
+	 **/
+	public abstract void execute(String jobName) throws Exception;
+
+	/**
 	 * Getter of the {@link JobGraphBuilder} of the streaming job.
 	 * 
 	 * @return jobgraph

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/076223cb/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
index 3be0c89..e07dfe5 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
@@ -80,7 +80,7 @@ public class WordCount {
 		}
 
 		// execute program
-		env.execute();
+		env.execute("Streaming WordCount");
 	}
 
 	// *************************************************************************


[05/12] git commit: [FLINK-1104] [streaming] Eliminated Tuple1 from connectors

Posted by mb...@apache.org.
[FLINK-1104] [streaming] Eliminated Tuple1 from connectors


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

Branch: refs/heads/master
Commit: 7cc24006a992d67cd3de54b867b38161f5b08d92
Parents: a3b0284
Author: ghermann <re...@gmail.com>
Authored: Sat Sep 20 13:55:13 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md                         | 66 ++++++++++----------
 .../connectors/kafka/KafkaTopology.java         | 31 +++++----
 .../function/source/GenSequenceFunction.java    |  4 +-
 3 files changed, 48 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7cc24006/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index e2f5854..f1b4189 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -456,7 +456,7 @@ An abstract class providing an interface for receiving data from Kafka. By imple
  * Write a deserializer function which processes the data coming from Kafka,
  * Stop the source manually when necessary with one of the close functions.
 
-The implemented class must extend `KafkaSource`, for example: `KafkaSource<Tuple1<String>>`.
+The implemented class must extend `KafkaSource`, for example: `KafkaSource<String>`.
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -472,12 +472,12 @@ An example of an implementation of a deserializer:
 
 ~~~java
 @Override
-public Tuple1<String> deserialize(byte[] msg) {
+public String deserialize(byte[] msg) {
 	String s = new String(msg);
 	if(s.equals("q")){
 		closeWithoutSend();
 	}
-	return new Tuple1<String>(s);
+	return new String(s);
 }
 ~~~
 
@@ -494,7 +494,7 @@ An abstract class providing an interface for sending data to Kafka. By implement
  * Write a serializer function to send data in the desired form to Kafka,
  * Stop the sink manually when necessary with one of the close functions.
 
-The implemented class must extend `KafkaSink`, for example `KafkaSink<Tuple1<String>, String>`.
+The implemented class must extend `KafkaSink`, for example `KafkaSink<String, String>`.
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -510,11 +510,11 @@ An example of an implementation of a serializer:
 
 ~~~java
 @Override
-public String serialize(Tuple1<String> tuple) {
-	if(tuple.f0.equals("q")){
+public String serialize(String tuple) {
+	if(tuple.equals("q")){
 		sendAndClose();
 	}
-	return tuple.f0;
+	return tuple;
 }
 ~~~
 
@@ -524,8 +524,8 @@ The API provided is the [same](#kafka_source_close) as the one for `KafkaSource`
 #### Building A Topology
 To use a Kafka connector as a source in Flink call the `addSource()` function with a new instance of the class which extends `KafkaSource` as parameter:
 
-~~~java
-DataStream<Tuple1<String>> stream1 = env.
+```java
+DataStream<String> stream1 = env.
 	addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
 	.print();
 ~~~
@@ -539,8 +539,8 @@ The followings have to be provided for the `MyKafkaSource()` constructor in orde
 
 Similarly to use a Kafka connector as a sink in Flink call the `addSink()` function with a new instance of the class which extends `KafkaSink`:
 
-~~~java
-DataStream<Tuple1<String>> stream2 = env
+```java
+DataStream<String> stream2 = env
 	.addSource(new MySource())
 	.addSink(new MyKafkaSink("test", "localhost:9092"));
 ~~~
@@ -567,7 +567,7 @@ An abstract class providing an interface for receiving data from Flume. By imple
  * Write a deserializer function which processes the data coming from Flume,
  * Stop the source manually when necessary with one of the close functions.
 
-The implemented class must extend `FlumeSource` for example: `FlumeSource<Tuple1<String>>`
+The implemented class must extend `FlumeSource` for example: `FlumeSource<String>`
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -583,10 +583,9 @@ An example of an implementation of a deserializer:
 
 ~~~java
 @Override
-public Tuple1<String> deserialize(byte[] msg) {
+public String deserialize(byte[] msg) {
 	String s = (String) SerializationUtils.deserialize(msg);
-	Tuple1<String> out = new Tuple1<String>();
-	out.f0 = s;
+	String out = s;
 	if (s.equals("q")) {
 		closeWithoutSend();
 	}
@@ -607,7 +606,7 @@ An abstract class providing an interface for sending data to Flume. By implement
 * Write a serializer function to send data in the desired form to Flume,
 * Stop the sink manually when necessary with one of the close functions.
 
-The implemented class must extend `FlumeSink`, for example `FlumeSink<Tuple1<String>, String>`.
+The implemented class must extend `FlumeSink`, for example `FlumeSink<String, String>`.
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -623,8 +622,8 @@ An example of an implementation of a serializer.
 
 ~~~java
 @Override
-public byte[] serialize(Tuple1<String> tuple) {
-	if (tuple.f0.equals("q")) {
+public byte[] serialize(String tuple) {
+	if (tuple.equals("q")) {
 		try {
 			sendAndClose();
 		} catch (Exception e) {
@@ -632,7 +631,7 @@ public byte[] serialize(Tuple1<String> tuple) {
 				+ host, e);
 		}
 	}
-	return SerializationUtils.serialize(tuple.f0);
+	return SerializationUtils.serialize(tuple);
 }
 ~~~
 
@@ -642,8 +641,8 @@ The API provided is the [same](#flume_source_close) as the one for `FlumeSource`
 #### Building A Topology
 To use a Flume connector as a source in Flink call the `addSource()` function with a new instance of the class which extends `FlumeSource` as parameter:
 
-~~~java
-DataStream<Tuple1<String>> dataStream1 = env
+```java
+DataStream<String> dataStream1 = env
 	.addSource(new MyFlumeSource("localhost", 41414))
 	.print();
 ~~~
@@ -655,8 +654,8 @@ The followings have to be provided for the `MyFlumeSource()` constructor in orde
 
 Similarly to use a Flume connector as a sink in Flink call the `addSink()` function with a new instance of the class which extends `FlumeSink`
 
-~~~java
-DataStream<Tuple1<String>> dataStream2 = env
+```java
+DataStream<String> dataStream2 = env
 	.fromElements("one", "two", "three", "four", "five", "q")
 	.addSink(new MyFlumeSink("localhost", 42424));
 ~~~
@@ -713,7 +712,7 @@ An abstract class providing an interface for receiving data from RabbitMQ. By im
 * Write a deserializer function which processes the data coming from RabbitMQ,
 * Stop the source manually when necessary with one of the close functions.
 
-The implemented class must extend `RabbitMQSource` for example: `RabbitMQSource<Tuple1<String>>`
+The implemented class must extend `RabbitMQSource` for example: `RabbitMQSource<String>`
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -729,10 +728,9 @@ An example of an implemetation of a deserializer:
 
 ~~~java
 @Override
-public Tuple1<String> deserialize(byte[] t) {
+public String deserialize(byte[] t) {
 	String s = (String) SerializationUtils.deserialize(t);
-	Tuple1<String> out = new Tuple1<String>();
-	out.f0 = s;
+	String out = s;
 	if (s.equals("q")) {
 		closeWithoutSend();
 	}
@@ -755,7 +753,7 @@ An abstract class providing an interface for sending data to RabbitMQ. By implem
 * Write a serializer function to send data in the desired form to RabbitMQ
 * Stop the sink manually when necessary with one of the close functions
 
-The implemented class must extend `RabbitMQSink` for example: `RabbitMQSink<Tuple1<String>, String>`
+The implemented class must extend `RabbitMQSink` for example: `RabbitMQSink<String, String>`
 
 ##### Constructor
 An example of an implementation of a constructor:
@@ -787,7 +785,7 @@ To use a RabbitMQ connector as a source in Flink call the `addSource()` function
 
 ~~~java
 @SuppressWarnings("unused")
-DataStream<Tuple1<String>> dataStream1 = env
+DataStream<String> dataStream1 = env
 	.addSource(new MyRMQSource("localhost", "hello"))
 	.print();
 ~~~
@@ -799,8 +797,8 @@ The followings have to be provided for the `MyRabbitMQSource()` constructor in o
 
 Similarly to use a RabbitMQ connector as a sink in Flink call the `addSink()` function with a new instance of the class which extends `RabbitMQSink`
 
-~~~java
-DataStream<Tuple1<String>> dataStream2 = env
+```java
+DataStream<String> dataStream2 = env
 	.fromElements("one", "two", "three", "four", "five", "q")
 	.addSink(new MyRMQSink("localhost", "hello"));
 ~~~
@@ -835,7 +833,7 @@ To run the container type:
 sudo docker run -p 127.0.0.1:5672:5672 -t -i flinkstreaming/flink-connectors-rabbitmq
 ~~~
 
-Now a terminal started running from the image with all the necessary configurations to test run the RabbitMQ connector. The -p flag binds the localhost's and the Docker container's port so RabbitMQ can communicate with the application through this.
+Now a terminal started running from the image with all the necessary configurations to test run the RabbitMQ connector. The -p flag binds the localhost's and the Docker container's ports so RabbitMQ can communicate with the application through these.
 
 To start the RabbitMQ server:
 
@@ -877,7 +875,7 @@ To run the container type:
 sudo docker run -p 127.0.0.1:2181:2181 -p 127.0.0.1:9092:9092 -t -i flinkstreaming/flink-connectors-kafka
 ~~~
 
-Now a terminal started running from the image with all the necessary configurations to test run the Kafka connector. The -p flag binds the localhost's and the Docker container's port so Kafka can communicate with the application through this.
+Now a terminal started running from the image with all the necessary configurations to test run the Kafka connector. The -p flag binds the localhost's and the Docker container's ports so Kafka can communicate with the application through these.
 First start a zookeeper in the background:
 
 ~~~batch
@@ -931,7 +929,7 @@ To run the container type:
 sudo docker run -t -i flinkstreaming/flink-connectors-flume
 ~~~
 
-Now a terminal started running from the image with all the necessary configurations to test run the Flume connector. The -p flag binds the localhost's and the Docker container's port so flume can communicate with the application through this.
+Now a terminal started running from the image with all the necessary configurations to test run the Flume connector. The -p flag binds the localhost's and the Docker container's ports so flume can communicate with the application through these.
 
 To have the latest version of Flink type:
 ~~~batch

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7cc24006/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
index 79f0b74..089efad 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTopology.java
@@ -17,7 +17,6 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.function.sink.SinkFunction;
@@ -29,20 +28,20 @@ import org.slf4j.LoggerFactory;
 public class KafkaTopology {
 	private static final Logger LOG = LoggerFactory.getLogger(KafkaTopology.class);
 	
-	public static final class MySource implements SourceFunction<Tuple1<String>> {
+	public static final class MySource implements SourceFunction<String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public void invoke(Collector<Tuple1<String>> collector) throws Exception {
+		public void invoke(Collector<String> collector) throws Exception {
 			for (int i = 0; i < 10; i++) {
-				collector.collect(new Tuple1<String>(Integer.toString(i)));
+				collector.collect(new String(Integer.toString(i)));
 			}
-			collector.collect(new Tuple1<String>("q"));
+			collector.collect(new String("q"));
 
 		}
 	}
 
-	public static final class MyKafkaSource extends KafkaSource<Tuple1<String>> {
+	public static final class MyKafkaSource extends KafkaSource<String> {
 		private static final long serialVersionUID = 1L;
 
 		public MyKafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
@@ -50,17 +49,17 @@ public class KafkaTopology {
 		}
 
 		@Override
-		public Tuple1<String> deserialize(byte[] msg) {
+		public String deserialize(byte[] msg) {
 			String s = new String(msg);
 			if (s.equals("q")) {
 				closeWithoutSend();
 			}
-			return new Tuple1<String>(s);
+			return new String(s);
 		}
 
 	}
 
-	public static final class MyKafkaSink extends KafkaSink<Tuple1<String>, String> {
+	public static final class MyKafkaSink extends KafkaSink<String, String> {
 		private static final long serialVersionUID = 1L;
 
 		public MyKafkaSink(String topicId, String brokerAddr) {
@@ -68,20 +67,20 @@ public class KafkaTopology {
 		}
 
 		@Override
-		public String serialize(Tuple1<String> tuple) {
-			if (tuple.f0.equals("q")) {
+		public String serialize(String tuple) {
+			if (tuple.equals("q")) {
 				sendAndClose();
 			}
-			return tuple.f0;
+			return tuple;
 		}
 
 	}
 	
-	public static final class MyKafkaPrintSink implements SinkFunction<Tuple1<String>> {
+	public static final class MyKafkaPrintSink implements SinkFunction<String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public void invoke(Tuple1<String> value) {
+		public void invoke(String value) {
 			if (LOG.isInfoEnabled()) {
 				LOG.info("String: <{}> arrived from Kafka", value);
 			}
@@ -95,12 +94,12 @@ public class KafkaTopology {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		@SuppressWarnings("unused")
-		DataStream<Tuple1<String>> stream1 = env
+		DataStream<String> stream1 = env
 			.addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
 			.addSink(new MyKafkaPrintSink());
 
 		@SuppressWarnings("unused")
-		DataStream<Tuple1<String>> stream2 = env
+		DataStream<String> stream2 = env
 			.addSource(new MySource())
 			.addSink(new MyKafkaSink("test", "localhost:9092"));
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/7cc24006/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
index acd63c2..69601ff 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/GenSequenceFunction.java
@@ -17,8 +17,7 @@
 
 package org.apache.flink.streaming.api.function.source;
 
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.util.Collector;
+import org.apache.flink.util.Collector;
 
 /**
  * Source Function used to generate the number sequence
@@ -30,7 +29,6 @@ public class GenSequenceFunction implements SourceFunction<Long> {
 
 	long from;
 	long to;
-	Tuple1<Long> outTuple = new Tuple1<Long>();
 
 	public GenSequenceFunction(long from, long to) {
 		this.from = from;


[09/12] git commit: [doc] [streaming] Added Twitter connector & Projection operator to docs

Posted by mb...@apache.org.
[doc] [streaming] Added Twitter connector & Projection operator to docs


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

Branch: refs/heads/master
Commit: 2dc5437a288d80dab5e10571c1bed630de79e1d3
Parents: 1bca326
Author: mbalassi <ba...@gmail.com>
Authored: Wed Sep 24 16:25:11 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md | 365 +++++++++++++++++++++++++------------------
 1 file changed, 216 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2dc5437a/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index f1b4189..27a32ba 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -22,9 +22,9 @@ Add the following dependency to your `pom.xml` to use the Flink Streaming.
 
 ~~~xml
 <dependency>
-	<groupId>org.apache.flink</groupId>
-	<artifactId>flink-streaming-core</artifactId>
-	<version>{{site.FLINK_VERSION_STABLE}}</version>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-streaming-core</artifactId>
+    <version>{{site.FLINK_VERSION_STABLE}}</version>
 </dependency>
 ~~~
 
@@ -39,32 +39,31 @@ The following program is a complete, working example of streaming WordCount. You
 ~~~java
 public class StreamingWordCount {
 
-	
-	public static void main(String[] args) {
+    public static void main(String[] args) {
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
-		
-		DataStream<Tuple2<String, Integer>> dataStream = env
-				.fromElements("Who's there?",
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+        
+        DataStream<Tuple2<String, Integer>> dataStream = env
+                .fromElements("Who's there?",
             "I think I hear them. Stand, ho! Who's there?")
-				.flatMap(new Splitter())
-				.groupBy(0)
-				.sum(1);
-		
-		dataStream.print();
-		
-		env.execute();
-	}
-	
-	public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		@Override
-		public void flatMap(String sentence, Collector<Tuple2<String, Integer>> out) throws Exception {
-			for (String word: sentence.split(" ")) {
-				out.collect(new Tuple2<String, Integer>(word, 1));
-			}
-		}
-	}
-	
+                .flatMap(new Splitter())
+                .groupBy(0)
+                .sum(1);
+        
+        dataStream.print();
+        
+        env.execute();
+    }
+    
+    public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
+        @Override
+        public void flatMap(String sentence, Collector<Tuple2<String, Integer>> out) throws Exception {
+            for (String word: sentence.split(" ")) {
+                out.collect(new Tuple2<String, Integer>(word, 1));
+            }
+        }
+    }
+    
 }
 ~~~
 
@@ -156,7 +155,8 @@ Besides the pre-defined solutions the user can implement their own source by imp
 
 ### Sinks
 
-`DataStreamSink` represents the different outputs of a Flink Streaming program. Every `DataStream` in a streaming program needs to be either transformed or closed down with a sink. There are several pre-defined implementations `DataStreamSink` available right away:
+`DataStreamSink` represents the different outputs of a Flink Streaming program. There are several pre-defined implementations `DataStreamSink` available right away:
+
  * `dataStream.print()` – Writes the DataStream to the standard output, practical for testing purposes
  * `dataStream.writeAsText(parameters)` – Writes the DataStream to a text file
  * `dataStream.writeAsCsv(parameters)` – Writes the DataStream to CSV format
@@ -180,11 +180,11 @@ A map operator that doubles the values of the input stream:
 
 ~~~java
 dataStream.map(new MapFunction<Integer, Integer>() {
-			@Override
-			public Integer map(Integer value) throws Exception {
-				return 2 * value;
-			}
-		})
+            @Override
+            public Integer map(Integer value) throws Exception {
+                return 2 * value;
+            }
+        })
 ~~~
 
 #### FlatMap
@@ -193,13 +193,13 @@ A flatmap operator that splits sentences to words:
 
 ~~~java
 dataStream.flatMap(new FlatMapFunction<String, String>() {
-			@Override
-			public void flatMap(String value, Collector<String> out) throws Exception {
-				for(String word: value.split(" ")){
-					out.collect(word);
-				}
-			}
-		})
+            @Override
+            public void flatMap(String value, Collector<String> out) throws Exception {
+                for(String word: value.split(" ")){
+                    out.collect(word);
+                }
+            }
+        })
 ~~~
 
 #### Filter
@@ -207,12 +207,12 @@ The Filter transformation applies a user-defined `FilterFunction` on each elemen
 A filter that filters out zero values:
 
 ~~~java
-dataStream.filter(new FilterFunction<Integer>() {		
-			@Override
-			public boolean filter(Integer value) throws Exception {
-				return value != 0;
-			}
-		})
+dataStream.filter(new FilterFunction<Integer>() { 
+            @Override
+            public boolean filter(Integer value) throws Exception {
+                return value != 0;
+            }
+        })
 ~~~
 
 #### Reduce
@@ -221,11 +221,11 @@ A reducer that sums up the incoming stream:
 
 ~~~java
 dataStream.reduce(new ReduceFunction<Integer>() {
-			@Override
-			public Integer reduce(Integer value1, Integer value2) throws Exception {
-				return value1+value2;
-			}
-		})
+            @Override
+            public Integer reduce(Integer value1, Integer value2) throws Exception {
+                return value1+value2;
+            }
+        })
 ~~~
 
 #### Merge
@@ -286,20 +286,20 @@ A CoMap operator that outputs true if an Integer value is received and false if
 ~~~java
 DataStream<Integer> dataStream1 = ...
 DataStream<String> dataStream2 = ...
-		
+        
 dataStream1.connect(dataStream2)
-	.map(new CoMapFunction<Integer, String, Boolean>() {
-			
-			@Override
-			public Boolean map1(Integer value) {
-				return true;
-			}
-			
-			@Override
-			public Boolean map2(String value) {
-				return false;
-			}
-		})
+    .map(new CoMapFunction<Integer, String, Boolean>() {
+            
+            @Override
+            public Boolean map1(Integer value) {
+                return true;
+            }
+            
+            @Override
+            public Boolean map2(String value) {
+                return false;
+            }
+        })
 ~~~
 
 #### FlatMap on ConnectedDataStream
@@ -308,20 +308,20 @@ The FlatMap operator for the `ConnectedDataStream` works similarly to CoMap, but
 ~~~java
 DataStream<Integer> dataStream1 = ...
 DataStream<String> dataStream2 = ...
-		
+        
 dataStream1.connect(dataStream2)
-	.flatMap(new CoFlatMapFunction<Integer, String, Boolean>() {
+    .flatMap(new CoFlatMapFunction<Integer, String, Boolean>() {
 
-			@Override
-			public void flatMap1(Integer value, Collector<Boolean> out) {
-				out.collect(true);
-			}
+            @Override
+            public void flatMap1(Integer value, Collector<Boolean> out) {
+                out.collect(true);
+            }
 
-			@Override
-			public void flatMap2(String value, Collector<Boolean> out) {
-				out.collect(false);
-			}
-		})
+            @Override
+            public void flatMap2(String value, Collector<Boolean> out) {
+                out.collect(false);
+            }
+        })
 ~~~
 
 #### Reduce on ConnectedDataStream
@@ -431,7 +431,7 @@ operator.setMutability(isMutable)
 ~~~
 
 [Back to top](#top)
-	
+    
 Stream connectors
 ----------------
 
@@ -452,6 +452,7 @@ This connector provides access to data streams from [Apache Kafka](https://kafka
 
 #### Kafka Source
 An abstract class providing an interface for receiving data from Kafka. By implementing the user must:
+
  * Write a constructor calling the constructor of the abstract class,
  * Write a deserializer function which processes the data coming from Kafka,
  * Stop the source manually when necessary with one of the close functions.
@@ -463,7 +464,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 public MyKafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
-	super(zkQuorum, groupId, topicId, numThreads);
+    super(zkQuorum, groupId, topicId, numThreads);
 }
 ~~~
 
@@ -473,11 +474,11 @@ An example of an implementation of a deserializer:
 ~~~java
 @Override
 public String deserialize(byte[] msg) {
-	String s = new String(msg);
-	if(s.equals("q")){
-		closeWithoutSend();
-	}
-	return new String(s);
+    String s = new String(msg);
+    if(s.equals("q")){
+        closeWithoutSend();
+    }
+    return new String(s);
 }
 ~~~
 
@@ -490,6 +491,7 @@ In the example provided `closeWithoutSend()` is used because here the String `"q
 
 #### Kafka Sink
 An abstract class providing an interface for sending data to Kafka. By implementing the user must:
+
  * Write a constructor calling the constructor of the abstract class,
  * Write a serializer function to send data in the desired form to Kafka,
  * Stop the sink manually when necessary with one of the close functions.
@@ -501,7 +503,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 public MyKafkaSink(String topicId, String brokerAddr) {
-	super(topicId, brokerAddr);
+    super(topicId, brokerAddr);
 }
 ~~~
 
@@ -511,10 +513,10 @@ An example of an implementation of a serializer:
 ~~~java
 @Override
 public String serialize(String tuple) {
-	if(tuple.equals("q")){
-		sendAndClose();
-	}
-	return tuple;
+    if(tuple.equals("q")){
+        sendAndClose();
+    }
+    return tuple;
 }
 ~~~
 
@@ -526,8 +528,8 @@ To use a Kafka connector as a source in Flink call the `addSource()` function wi
 
 ```java
 DataStream<String> stream1 = env.
-	addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
-	.print();
+    addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
+    .print();
 ~~~
 
 The followings have to be provided for the `MyKafkaSource()` constructor in order:
@@ -541,8 +543,8 @@ Similarly to use a Kafka connector as a sink in Flink call the `addSink()` funct
 
 ```java
 DataStream<String> stream2 = env
-	.addSource(new MySource())
-	.addSink(new MyKafkaSink("test", "localhost:9092"));
+    .addSource(new MySource())
+    .addSink(new MyKafkaSink("test", "localhost:9092"));
 ~~~
 
 The followings have to be provided for the `MyKafkaSink()` constructor in order:
@@ -563,6 +565,7 @@ This connector provides access to datastreams from [Apache Flume](http://flume.a
 
 #### Flume Source
 An abstract class providing an interface for receiving data from Flume. By implementing the user must:
+
  * Write a constructor calling the constructor of the abstract class,
  * Write a deserializer function which processes the data coming from Flume,
  * Stop the source manually when necessary with one of the close functions.
@@ -574,7 +577,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 MyFlumeSource(String host, int port) {
-	super(host, port);
+    super(host, port);
 }
 ~~~
 
@@ -584,12 +587,12 @@ An example of an implementation of a deserializer:
 ~~~java
 @Override
 public String deserialize(byte[] msg) {
-	String s = (String) SerializationUtils.deserialize(msg);
-	String out = s;
-	if (s.equals("q")) {
-		closeWithoutSend();
-	}
-	return out;
+    String s = (String) SerializationUtils.deserialize(msg);
+    String out = s;
+    if (s.equals("q")) {
+        closeWithoutSend();
+    }
+    return out;
 }
 ~~~
 
@@ -602,6 +605,7 @@ In the example `closeWithoutSend()` is used because here the String `"q"` is met
 
 #### Flume Sink
 An abstract class providing an interface for sending data to Flume. By implementing the user must:
+
 * Write a constructor calling the constructor of the abstract class,
 * Write a serializer function to send data in the desired form to Flume,
 * Stop the sink manually when necessary with one of the close functions.
@@ -613,7 +617,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 public MyFlumeSink(String host, int port) {
-	super(host, port);
+    super(host, port);
 }
 ~~~
 
@@ -623,15 +627,15 @@ An example of an implementation of a serializer.
 ~~~java
 @Override
 public byte[] serialize(String tuple) {
-	if (tuple.equals("q")) {
-		try {
-			sendAndClose();
-		} catch (Exception e) {
-			new RuntimeException("Error while closing Flume connection with " + port + " at "
-				+ host, e);
-		}
-	}
-	return SerializationUtils.serialize(tuple);
+    if (tuple.equals("q")) {
+        try {
+            sendAndClose();
+        } catch (Exception e) {
+            new RuntimeException("Error while closing Flume connection with " + port + " at "
+                + host, e);
+        }
+    }
+    return SerializationUtils.serialize(tuple);
 }
 ~~~
 
@@ -643,8 +647,8 @@ To use a Flume connector as a source in Flink call the `addSource()` function wi
 
 ```java
 DataStream<String> dataStream1 = env
-	.addSource(new MyFlumeSource("localhost", 41414))
-	.print();
+    .addSource(new MyFlumeSource("localhost", 41414))
+    .print();
 ~~~
 
 The followings have to be provided for the `MyFlumeSource()` constructor in order:
@@ -656,8 +660,8 @@ Similarly to use a Flume connector as a sink in Flink call the `addSink()` funct
 
 ```java
 DataStream<String> dataStream2 = env
-	.fromElements("one", "two", "three", "four", "five", "q")
-	.addSink(new MyFlumeSink("localhost", 42424));
+    .fromElements("one", "two", "three", "four", "five", "q")
+    .addSink(new MyFlumeSink("localhost", 42424));
 ~~~
 
 The followings have to be provided for the `MyFlumeSink()` constructor in order:
@@ -719,7 +723,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 public MyRMQSource(String HOST_NAME, String QUEUE_NAME) {
-	super(HOST_NAME, QUEUE_NAME);
+    super(HOST_NAME, QUEUE_NAME);
 }
 ~~~
 
@@ -729,12 +733,12 @@ An example of an implemetation of a deserializer:
 ~~~java
 @Override
 public String deserialize(byte[] t) {
-	String s = (String) SerializationUtils.deserialize(t);
-	String out = s;
-	if (s.equals("q")) {
-		closeWithoutSend();
-	}
-	return out;
+    String s = (String) SerializationUtils.deserialize(t);
+    String out = s;
+    if (s.equals("q")) {
+        closeWithoutSend();
+    }
+    return out;
 }
 ~~~
 
@@ -749,6 +753,7 @@ In the example `closeWithoutSend()` is used because here the String `"q"` is met
 
 #### RabbitMQ Sink
 An abstract class providing an interface for sending data to RabbitMQ. By implementing the user must:
+
 * Write a constructor calling the constructor of the abstract class
 * Write a serializer function to send data in the desired form to RabbitMQ
 * Stop the sink manually when necessary with one of the close functions
@@ -760,7 +765,7 @@ An example of an implementation of a constructor:
 
 ~~~java
 public MyRMQSink(String HOST_NAME, String QUEUE_NAME) {
-	super(HOST_NAME, QUEUE_NAME);
+    super(HOST_NAME, QUEUE_NAME);
 }
 ~~~
 
@@ -770,10 +775,10 @@ An example of an implementation of a serializer.
 ~~~java
 @Override
 public byte[] serialize(Tuple tuple) {
-	if (t.getField(0).equals("q")) {
-		sendAndClose();
-	}
-	return SerializationUtils.serialize(tuple.f0);
+    if (t.getField(0).equals("q")) {
+        sendAndClose();
+    }
+    return SerializationUtils.serialize(tuple.f0);
 }
 ~~~
 
@@ -784,10 +789,9 @@ The API provided is the [same](#rmq_source_close) as the one for `RabbitMQSource
 To use a RabbitMQ connector as a source in Flink call the `addSource()` function with a new instance of the class which extends `RabbitMQSource` as parameter:
 
 ~~~java
-@SuppressWarnings("unused")
 DataStream<String> dataStream1 = env
-	.addSource(new MyRMQSource("localhost", "hello"))
-	.print();
+    .addSource(new MyRMQSource("localhost", "hello"))
+    .print();
 ~~~
 
 The followings have to be provided for the `MyRabbitMQSource()` constructor in order:
@@ -799,8 +803,8 @@ Similarly to use a RabbitMQ connector as a sink in Flink call the `addSink()` fu
 
 ```java
 DataStream<String> dataStream2 = env
-	.fromElements("one", "two", "three", "four", "five", "q")
-	.addSink(new MyRMQSink("localhost", "hello"));
+    .fromElements("one", "two", "three", "four", "five", "q")
+    .addSink(new MyRMQSink("localhost", "hello"));
 ~~~
 
 The followings have to be provided for the `MyRabbitMQSink()` constructor in order:
@@ -812,13 +816,79 @@ More about RabbitMQ can be found [here](http://www.rabbitmq.com/).
 
 [Back to top](#top)
 
+### Twitter Streaming API
+
+Twitter Streaming API provides opportunity to connect to the stream of tweets made available by Twitter. Flink Streaming comes with a built-in `TwitterSource` class for establishing a connection to this stream.
+
+#### Authentication
+In order to connect to Twitter stream the user has to register their program and acquire the necessary information for the authentication. The process is described below.
+
+#### Acquiring the authentication information
+First of all, a Twitter account is needed. Sign up for free at [twitter.com/signup](https://twitter.com/signup) or sing in at Twitter's [Application Management](https://apps.twitter.com/) and register the application by clicking on the "Create New App" button. Fill out a form about your program and accept the Terms and Conditions. 
+After selecting the application you the API key and API secret (called `consumerKey` and `sonsumerSecret` in `TwitterSource` respectively) is located on the "API Keys" tab. The necessary access token data (`token` and `secret`) can be acquired here. 
+Remember to keep these pieces of information a secret and do not push them to public repositories.
+
+#### Accessing the authentication information
+Create a properties file and pass its path in the constructor of `TwitterSource`. The content of the file should be similar to this:
+
+~~~batch
+#properties file for my app
+secret=***
+consumerSecret=***
+token=***-***
+consumerKey=***
+~~~
+
+#### Constructors
+The `TwitterSource` class has two constructors.
+
+1. `public TwitterSource(String authPath, int numberOfTweets);` 
+to emit finite number of tweets
+2. `public TwitterSource(String authPath);` 
+for streaming
+
+Both constructors expect a `String authPath` argument determining the location of the properties file containing the authentication information. In the first case, `numberOfTweets` determine how many tweet the source emits. 
+
+#### Usage
+In constract to other connecters the `TwitterSource` depends on no additional services. For example the following code should run gracefully:
+
+~~~java
+DataStream<String> streamSource = env.AddSource(new TwitterSource("/PATH/TO/myFile.properties"));
+~~~
+
+The `TwitterSource` emits strings containing a JSON code. 
+To retrieve information from the JSON code you can add a FlatMap or a Map function handling JSON code. For example use an implementation `JSONParseFlatMap` abstract class among the examples. `JSONParseFlatMap` is an extension of the `FlatMapFunction` and has a
+
+~~~java
+String getField(String jsonText, String field);
+~~~
+
+function which can be use to acquire the value of a given field. 
+
+There are two basic types of tweets. The usual tweets contain information such as date and time of creation, id, user, language and many more details. The other type is the delete information.
+
+#### Example
+`TwitterLocal` is an example how to use `TwitterSource`. It implements a language frequency counter program. 
+
+[Back to top](#top)
+
 ### Docker containers for connectors<a name="docker_connectors"></a>
 
 A Docker container is provided with all the required configurations for test running the connectors of Apache Flink. The servers for the message queues will be running on the docker container while the example topology can be run on the user's computer. The only exception is Flume, more can be read about this issue at the [Flume section](#flume). 
 
 #### Installing Docker
 The official Docker installation guide can be found [here](https://docs.docker.com/installation/).
-After installing Docker an image can be pulled for each connector. Containers can be started from these images where all the required configurations are set. 
+After installing Docker an image can be pulled for each connector. Containers can be started from these images where all the required configurations are set.
+
+#### Creating a jar with all the dependencies
+For the easiest set up create a jar with all the dependencies of the *flink-streaming-connectors* project.
+
+~~~batch
+cd /PATH/TO/GIT/incubator-flink/flink-addons/flink-streaming-connectors
+mvn assembly:assembly
+~~~batch
+
+This creates an assembly jar under *flink-streaming-connectors/target*. 
 
 #### RabbitMQ
 Pull the image:
@@ -844,14 +914,11 @@ sudo /etc/init.d/rabbitmq-server start
 To launch the example on the host computer execute:
 
 ~~~batch
-java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.rabbitmq.RMQTopology > log.txt 2> errorlog.txt
+java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.rabbitmq.RMQTopology \
+> log.txt 2> errorlog.txt
 ~~~
 
-The maven assemby plugin creates one jar with all the requiered dependencies. If the project is in a directory called git then the jar can be found here: (the version number may change later)
-~~~batch
-/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ~~~
-
-In the example there are to connectors. One that sends messages to RabbitMQ and one that receives messages from the same queue. In the logger messages the arriving messages can be observed in the following format:
+In the example there are two connectors. One that sends messages to RabbitMQ and one that receives messages from the same queue. In the logger messages the arriving messages can be observed in the following format:
 
 ~~~
 <DATE> INFO rabbitmq.RMQTopology: String: <one> arrived from RMQ 
@@ -872,33 +939,34 @@ sudo docker pull flinkstreaming/flink-connectors-kafka
 To run the container type:
 
 ~~~batch
-sudo docker run -p 127.0.0.1:2181:2181 -p 127.0.0.1:9092:9092 -t -i flinkstreaming/flink-connectors-kafka
+sudo docker run -p 127.0.0.1:2181:2181 -p 127.0.0.1:9092:9092 -t -i \
+flinkstreaming/flink-connectors-kafka
 ~~~
 
 Now a terminal started running from the image with all the necessary configurations to test run the Kafka connector. The -p flag binds the localhost's and the Docker container's ports so Kafka can communicate with the application through these.
 First start a zookeeper in the background:
 
 ~~~batch
-/kafka_2.9.2-0.8.1.1/bin/zookeeper-server-start.sh /kafka_2.9.2-0.8.1.1/config/zookeeper.properties > zookeeperlog.txt &
+/kafka_2.9.2-0.8.1.1/bin/zookeeper-server-start.sh /kafka_2.9.2-0.8.1.1/config/zookeeper.properties \
+> zookeeperlog.txt &
 ~~~
 
 Then start the kafka server in the background:
 
 ~~~batch
-/kafka_2.9.2-0.8.1.1/bin/kafka-server-start.sh /kafka_2.9.2-0.8.1.1/config/server.properties > serverlog.txt 2> servererr.txt &
+/kafka_2.9.2-0.8.1.1/bin/kafka-server-start.sh /kafka_2.9.2-0.8.1.1/config/server.properties \
+ > serverlog.txt 2> servererr.txt &
 ~~~
 
 To launch the example on the host computer execute:
 
 ~~~batch
-java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.kafka.KafkaTopology > log.txt 2> errorlog.txt
+java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.kafka.KafkaTopology \
+> log.txt 2> errorlog.txt
 ~~~
 
-The maven assemby plugin creates one jar with all the requiered dependencies. If the project is in a directory called git then the jar can be found here: (the version number may change later)
-~~~batch
-/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ~~~
 
-In the example there are to connectors. One that sends messages to Kafka and one that receives messages from the same queue. In the logger messages the arriving messages can be observed in the following format:
+In the example there are two connectors. One that sends messages to Kafka and one that receives messages from the same queue. In the logger messages the arriving messages can be observed in the following format:
 
 ~~~
 <DATE> INFO kafka.KafkaTopology: String: (0) arrived from Kafka
@@ -915,7 +983,7 @@ In the example there are to connectors. One that sends messages to Kafka and one
 
 #### Apache Flume
 
-At the moment remote access for Flume connectors does not work. This example can be run only on the same machine where the Flume server is. In this case both will be in the Docker container.
+At the moment remote access for Flume connectors does not work. This example is only runnable on the same machine where the Flume server is. In this case both will be in the Docker container.
 
 Pull the image:
 
@@ -947,7 +1015,9 @@ mvn install -DskipTests
 First start the server in the background:
 
 ~~~batch
-/apache-flume-1.5.0-bin/bin/flume-ng agent --conf conf --conf-file /apache-flume-1.5.0-bin/example.conf --name a1 -Dflume.root.logger=INFO,console > /flumelog.txt 2> /flumeerr.txt &
+/apache-flume-1.5.0-bin/bin/flume-ng agent \
+--conf conf --conf-file /apache-flume-1.5.0-bin/example.conf --name a1 \
+-Dflume.root.logger=INFO,console > /flumelog.txt 2> /flumeerr.txt &
 ~~~
 
 Then press enter and launch the example with:
@@ -956,9 +1026,6 @@ Then press enter and launch the example with:
 java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.flume.FlumeTopology
 ~~~
 
-The maven assemby plugin creates one jar with all the requiered dependencies. If the project is in a directory called git then the jar can be found here: (the version number may change later)
-~~~batch
-/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ~~~
 In the example there are to connectors. One that sends messages to Flume and one that receives messages from the same queue. In the logger messages the arriving messages can be observed in the following format:
 
 ~~~


[12/12] git commit: [streaming] Streaming packages added to flink-dist lib

Posted by mb...@apache.org.
[streaming] Streaming packages added to flink-dist lib


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

Branch: refs/heads/master
Commit: cb81319d9b2ebf719c4832997797aa2907a3e5bb
Parents: 076223c
Author: mbalassi <ba...@gmail.com>
Authored: Wed Sep 24 21:19:36 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 21:19:36 2014 +0200

----------------------------------------------------------------------
 .../flink-streaming-examples/pom.xml            |  2 +-
 flink-dist/pom.xml                              | 36 +++++++++++++++-----
 2 files changed, 28 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb81319d/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
index 7222879..7c02f27 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
@@ -48,7 +48,7 @@ under the License.
 			<version>${project.version}</version>
 		</dependency>
 
-        <dependency>
+        	<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-connectors</artifactId>
 			<version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb81319d/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 0514b66..b9aee28 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -94,6 +94,24 @@ under the License.
 			<artifactId>flink-scala-examples</artifactId>
 			<version>${project.version}</version>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-core</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-connectors</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-examples</artifactId>
+			<version>${project.version}</version>
+		</dependency>
 	</dependencies>
 
 	<!-- See main pom.xml for explanation of profiles -->
@@ -110,7 +128,7 @@ under the License.
 				<!-- No extra dependencies: pact-hbase is currently not compatible with Hadoop v1 -->
 			</dependencies>
 		</profile>
-		
+
 		<profile>
 			<id>include-yarn</id>
 			<activation>
@@ -149,8 +167,8 @@ under the License.
 									<archive>
 										<manifest>
 											<mainClass>org.apache.flink.yarn.Client</mainClass>
-											<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>  
-											<addDefaultImplementationEntries>true</addDefaultImplementationEntries>  
+											<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
+											<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
 										</manifest>
 									</archive>
 									<descriptors>
@@ -158,7 +176,7 @@ under the License.
 									</descriptors>
 								</configuration>
 							</execution>
-							
+
 							<!-- yarn bin directory -->
 							<execution>
 								<id>yarn-bin</id>
@@ -179,7 +197,7 @@ under the License.
 				</plugins>
 			</build>
 		</profile>
-		
+
 		<profile>
 			<id>hadoop-2</id>
 			<activation>
@@ -196,7 +214,7 @@ under the License.
 				</dependency>
 			</dependencies>
 		</profile>
-		
+
 		<profile>
 			<id>debian-package</id>
 			<build>
@@ -336,8 +354,8 @@ under the License.
 			</build>
 		</profile>
 	</profiles>
-	
-	
+
+
 
 	<build>
 		<plugins>
@@ -368,7 +386,7 @@ under the License.
 					<skip>true</skip>
 				</configuration>
 			</plugin>
-			
+
 			<plugin>
 				<!-- Description: https://github.com/ktoso/maven-git-commit-id-plugin -->
 				<groupId>pl.project13.maven</groupId>


[10/12] git commit: [streaming] added batchReduceGroup, windowReduceGroup functionality to ConnectedDataStream and GroupedConnectedDataStream

Posted by mb...@apache.org.
[streaming] added batchReduceGroup, windowReduceGroup functionality to ConnectedDataStream and GroupedConnectedDataStream


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

Branch: refs/heads/master
Commit: ad983377dcc297a137f74a5895b8336e446ef864
Parents: 7337110
Author: szape <ne...@gmail.com>
Authored: Fri Sep 12 11:26:18 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Wed Sep 24 19:54:39 2014 +0200

----------------------------------------------------------------------
 .../api/datastream/ConnectedDataStream.java     | 249 ++++++++++++++++-
 .../datastream/GroupedConnectedDataStream.java  | 268 ++++++++++++++++++-
 .../co/CoBatchGroupReduceInvokable.java         |   2 +-
 .../flink/streaming/state/CircularFifoList.java |  15 +-
 .../flink/streaming/state/StreamIterator.java   |   5 +-
 5 files changed, 511 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ad983377/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
index 0499c4c..256f470 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
@@ -27,13 +27,19 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.function.co.CoFlatMapFunction;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
 import org.apache.flink.streaming.api.function.co.CoMapFunction;
 import org.apache.flink.streaming.api.function.co.CoReduceFunction;
 import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
+import org.apache.flink.streaming.api.function.co.RichCoReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoBatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoFlatMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoStreamReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoWindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
 
@@ -59,7 +65,7 @@ public class ConnectedDataStream<IN1, IN2> {
 		this.jobGraphBuilder = jobGraphBuilder;
 		this.environment = environment;
 		this.input1 = input1.copy();
-		this.input2 = input2.copy();		
+		this.input2 = input2.copy();
 	}
 
 	/**
@@ -82,20 +88,22 @@ public class ConnectedDataStream<IN1, IN2> {
 
 	/**
 	 * Gets the type of the first input
+	 * 
 	 * @return The type of the first input
 	 */
 	public TypeInformation<IN1> getInputType1() {
 		return input1.getOutputType();
 	}
-	
+
 	/**
 	 * Gets the type of the second input
+	 * 
 	 * @return The type of the second input
 	 */
 	public TypeInformation<IN2> getInputType2() {
 		return input2.getOutputType();
 	}
-	
+
 	/**
 	 * GroupBy operation for connected data stream. Groups the elements of
 	 * input1 and input2 according to keyPosition1 and keyPosition2. Used for
@@ -174,12 +182,15 @@ public class ConnectedDataStream<IN1, IN2> {
 	}
 
 	/**
-	 * Applies a reduce transformation on both input of a
-	 * {@link ConnectedDataStream} and maps the output to a common type. The
-	 * transformation calls {@link CoReduceFunction#reduce1} and
-	 * {@link CoReduceFunction#map1} for each element of the first input and
-	 * {@link CoReduceFunction#reduce2} and {@link CoReduceFunction#map2} for
-	 * each element of the second input.
+	 * Applies a reduce transformation on a {@link ConnectedDataStream} and maps
+	 * the outputs to a common type. The transformation calls
+	 * {@link CoReduceFunction#reduce1} and {@link CoReduceFunction#map1} for
+	 * each element of the first input and {@link CoReduceFunction#reduce2} and
+	 * {@link CoReduceFunction#map2} for each element of the second input. This
+	 * type of reduce is much faster than reduceGroup since the reduce function
+	 * can be applied incrementally. The user can also extend the
+	 * {@link RichCoReduceFunction} to gain access to other features provided by
+	 * the {@link RichFuntion} interface.
 	 * 
 	 * @param coReducer
 	 *            The {@link CoReduceFunction} that will be called for every
@@ -199,6 +210,226 @@ public class ConnectedDataStream<IN1, IN2> {
 				new CoStreamReduceInvokable<IN1, IN2, OUT>(coReducer));
 	}
 
+	/**
+	 * Applies a reduceGroup transformation on the preset batches of the inputs
+	 * of a {@link ConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each batch of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each batch of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * batch, the batch is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param batchSize1
+	 *            The number of elements in a batch of the first input.
+	 * @param batchSize2
+	 *            The number of elements in a batch of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> batchReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long batchSize1, long batchSize2) {
+		return batchReduceGroup(coReducer, batchSize1, batchSize2, batchSize1, batchSize2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset batches of the inputs
+	 * of a {@link ConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each batch of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each batch of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * batch, the batch is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param batchSize1
+	 *            The number of elements in a batch of the first input.
+	 * @param batchSize2
+	 *            The number of elements in a batch of the second input.
+	 * @param slideSize1
+	 *            The number of elements a batch of the first input is slid by.
+	 * @param slideSize2
+	 *            The number of elements a batch of the second input is slid by.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> batchReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long batchSize1, long batchSize2,
+			long slideSize1, long slideSize2) {
+
+		if (batchSize1 < 1 || batchSize2 < 1) {
+			throw new IllegalArgumentException("Batch size must be positive");
+		}
+		if (slideSize1 < 1 || slideSize2 < 1) {
+			throw new IllegalArgumentException("Slide size must be positive");
+		}
+		if (batchSize1 < slideSize1 || batchSize2 < slideSize2) {
+			throw new IllegalArgumentException("Batch size must be at least slide size");
+		}
+
+		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
+				CoGroupReduceFunction.class, 0);
+		FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coReducer,
+				CoGroupReduceFunction.class, 1);
+		FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coReducer,
+				CoGroupReduceFunction.class, 2);
+
+		return addCoFunction("coBatchReduce", coReducer, in1TypeWrapper, in2TypeWrapper,
+				outTypeWrapper, new CoBatchGroupReduceInvokable<IN1, IN2, OUT>(coReducer,
+						batchSize1, batchSize2, slideSize1, slideSize2));
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link ConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * window, the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, windowSize1, windowSize2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link ConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * window, the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param slideInterval1
+	 *            The time interval a window of the first input is slid by.
+	 * @param slideInterval2
+	 *            The time interval a window of the second input is slid by.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			long slideInterval1, long slideInterval2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, slideInterval1,
+				slideInterval2, new DefaultTimeStamp<IN1>(), new DefaultTimeStamp<IN2>());
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link ConnectedDataStream}, where the time is provided by
+	 * timestamps. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * window, the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param timestamp1
+	 *            The predefined timestamp function of the first input.
+	 * @param timestamp2
+	 *            The predefined timestamp function of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			TimeStamp<IN1> timestamp1, TimeStamp<IN2> timestamp2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, windowSize1, windowSize2,
+				timestamp1, timestamp2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link ConnectedDataStream}, where the time is provided by
+	 * timestamps. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. Each {@link CoGroupReduceFunction} call can return any number of
+	 * elements including none. When the reducer has ran for all the values of a
+	 * window, the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param slideInterval1
+	 *            The time interval a window of the first input is slid by.
+	 * @param slideInterval2
+	 *            The time interval a window of the second input is slid by.
+	 * @param timestamp1
+	 *            The predefined timestamp function of the first input.
+	 * @param timestamp2
+	 *            The predefined timestamp function of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			long slideInterval1, long slideInterval2, TimeStamp<IN1> timestamp1,
+			TimeStamp<IN2> timestamp2) {
+
+		if (windowSize1 < 1 || windowSize2 < 1) {
+			throw new IllegalArgumentException("Window size must be positive");
+		}
+		if (slideInterval1 < 1 || slideInterval2 < 1) {
+			throw new IllegalArgumentException("Slide interval must be positive");
+		}
+		if (windowSize1 < slideInterval1 || windowSize2 < slideInterval2) {
+			throw new IllegalArgumentException("Window size must be at least slide interval");
+		}
+
+		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
+				CoGroupReduceFunction.class, 0);
+		FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coReducer,
+				CoGroupReduceFunction.class, 1);
+		FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coReducer,
+				CoGroupReduceFunction.class, 2);
+
+		return addCoFunction("coWindowReduce", coReducer, in1TypeWrapper, in2TypeWrapper,
+				outTypeWrapper, new CoWindowGroupReduceInvokable<IN1, IN2, OUT>(coReducer,
+						windowSize1, windowSize2, slideInterval1, slideInterval2, timestamp1,
+						timestamp2));
+	}
+
 	protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,
 			final Function function, TypeSerializerWrapper<IN1> in1TypeWrapper,
 			TypeSerializerWrapper<IN2> in2TypeWrapper, TypeSerializerWrapper<OUT> outTypeWrapper,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ad983377/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedConnectedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedConnectedDataStream.java
index 59dd2fa..0b6a7b5 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedConnectedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedConnectedDataStream.java
@@ -19,8 +19,14 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
 import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+import org.apache.flink.streaming.api.function.co.RichCoReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedBatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedWindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 
 public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN1, IN2> {
@@ -37,20 +43,24 @@ public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN
 	}
 
 	/**
-	 * Applies a CoReduce transformation on a {@link ConnectedDataStream}
-	 * grouped by the given key position and maps the output to a common type.
-	 * The {@link CoReduceFunction} will receive input values based on the key
-	 * positions. The transformation calls {@link CoReduceFunction#reduce1} and
-	 * {@link CoReduceFunction#map1} for each element of the first input and
-	 * {@link CoReduceFunction#reduce2} and {@link CoReduceFunction#map2} for
-	 * each element of the second input. For each input, only values with the
-	 * same key will go to the same reducer.
+	 * Applies a reduce transformation on a {@link GroupedConnectedDataStream},
+	 * and maps the outputs to a common type. The transformation calls
+	 * {@link CoReduceFunction#reduce1} and {@link CoReduceFunction#map1} for
+	 * each element of the first input and {@link CoReduceFunction#reduce2} and
+	 * {@link CoReduceFunction#map2} for each element of the second input. For
+	 * both inputs, the reducer is applied on every group of elements sharing
+	 * the same key at the respective position. This type of reduce is much
+	 * faster than reduceGroup since the reduce function can be applied
+	 * incrementally. The user can also extend the {@link RichCoReduceFunction}
+	 * to gain access to other features provided by the {@link RichFuntion}
+	 * interface.
 	 * 
 	 * @param coReducer
 	 *            The {@link CoReduceFunction} that will be called for every
-	 *            element with the same key of each input DataStream.
-	 * @return The transformed DataStream.
+	 *            element of the inputs.
+	 * @return The transformed {@link DataStream}.
 	 */
+	@Override
 	public <OUT> SingleOutputStreamOperator<OUT, ?> reduce(CoReduceFunction<IN1, IN2, OUT> coReducer) {
 
 		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
@@ -64,4 +74,242 @@ public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN
 				new CoGroupedReduceInvokable<IN1, IN2, OUT>(coReducer, keyPosition1, keyPosition2));
 	}
 
+	/**
+	 * Applies a reduceGroup transformation on the preset batches of the inputs
+	 * of a {@link GroupedConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each batch of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each batch of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every batch sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a batch,
+	 * the batch is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param batchSize1
+	 *            The number of elements in a batch of the first input.
+	 * @param batchSize2
+	 *            The number of elements in a batch of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> batchReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long batchSize1, long batchSize2) {
+		return batchReduceGroup(coReducer, batchSize1, batchSize2, batchSize1, batchSize2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset batches of the inputs
+	 * of a {@link GroupedConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each batch of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each batch of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every batch sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a batch,
+	 * the batch is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param batchSize1
+	 *            The number of elements in a batch of the first input.
+	 * @param batchSize2
+	 *            The number of elements in a batch of the second input.
+	 * @param slideSize1
+	 *            The number of elements a batch of the first input is slid by.
+	 * @param slideSize2
+	 *            The number of elements a batch of the second input is slid by.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> batchReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long batchSize1, long batchSize2,
+			long slideSize1, long slideSize2) {
+
+		if (batchSize1 < 1 || batchSize2 < 1) {
+			throw new IllegalArgumentException("Batch size must be positive");
+		}
+		if (slideSize1 < 1 || slideSize2 < 1) {
+			throw new IllegalArgumentException("Slide size must be positive");
+		}
+		if (batchSize1 < slideSize1 || batchSize2 < slideSize2) {
+			throw new IllegalArgumentException("Batch size must be at least slide size");
+		}
+
+		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
+				CoGroupReduceFunction.class, 0);
+		FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coReducer,
+				CoGroupReduceFunction.class, 1);
+		FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coReducer,
+				CoGroupReduceFunction.class, 2);
+
+		return addCoFunction("coBatchReduce", coReducer, in1TypeWrapper, in2TypeWrapper,
+				outTypeWrapper, new CoGroupedBatchGroupReduceInvokable<IN1, IN2, OUT>(coReducer,
+						batchSize1, batchSize2, slideSize1, slideSize2, keyPosition1, keyPosition2));
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link GroupedConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every window sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a window,
+	 * the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, windowSize1, windowSize2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link GroupedConnectedDataStream}. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every window sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a window,
+	 * the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param slideInterval1
+	 *            The time interval a window of the first input is slid by.
+	 * @param slideInterval2
+	 *            The time interval a window of the second input is slid by.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			long slideInterval1, long slideInterval2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, slideInterval1,
+				slideInterval2, new DefaultTimeStamp<IN1>(), new DefaultTimeStamp<IN2>());
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link GroupedConnectedDataStream}, where the time is
+	 * provided by timestamps. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every window sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a window,
+	 * the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param timestamp1
+	 *            The predefined timestamp function of the first input.
+	 * @param timestamp2
+	 *            The predefined timestamp function of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			TimeStamp<IN1> timestamp1, TimeStamp<IN2> timestamp2) {
+		return windowReduceGroup(coReducer, windowSize1, windowSize2, windowSize1, windowSize2,
+				timestamp1, timestamp2);
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on the preset time windows of the
+	 * inputs of a {@link GroupedConnectedDataStream}, where the time is
+	 * provided by timestamps. The transformation calls
+	 * {@link CoGroupReduceFunction#reduce1} for each window of the first input
+	 * and {@link CoGroupReduceFunction#reduce2} for each window of the second
+	 * input. For both inputs, the reducer is applied on every group of elements
+	 * of every window sharing the same key at the respective position. Each
+	 * {@link CoGroupReduceFunction} call can return any number of elements
+	 * including none. When the reducer has ran for all the values of a window,
+	 * the window is slid forward. The user can also extend
+	 * {@link RichCoGroupReduceFunction} to gain access to other features
+	 * provided by the {@link RichFuntion} interface.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoGroupReduceFunction} that will be called for
+	 *            every batch of each input.
+	 * @param windowSize1
+	 *            The size of the time window of the first input.
+	 * @param windowSize2
+	 *            The size of the time window of the second input.
+	 * @param slideInterval1
+	 *            The time interval a window of the first input is slid by.
+	 * @param slideInterval2
+	 *            The time interval a window of the second input is slid by.
+	 * @param timestamp1
+	 *            The predefined timestamp function of the first input.
+	 * @param timestamp2
+	 *            The predefined timestamp function of the second input.
+	 * @return The transformed {@link DataStream}.
+	 */
+	@Override
+	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReducer, long windowSize1, long windowSize2,
+			long slideInterval1, long slideInterval2, TimeStamp<IN1> timestamp1,
+			TimeStamp<IN2> timestamp2) {
+
+		if (windowSize1 < 1 || windowSize2 < 1) {
+			throw new IllegalArgumentException("Window size must be positive");
+		}
+		if (slideInterval1 < 1 || slideInterval2 < 1) {
+			throw new IllegalArgumentException("Slide interval must be positive");
+		}
+		if (windowSize1 < slideInterval1 || windowSize2 < slideInterval2) {
+			throw new IllegalArgumentException("Window size must be at least slide interval");
+		}
+
+		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
+				CoGroupReduceFunction.class, 0);
+		FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coReducer,
+				CoGroupReduceFunction.class, 1);
+		FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coReducer,
+				CoGroupReduceFunction.class, 2);
+
+		return addCoFunction("coWindowReduce", coReducer, in1TypeWrapper, in2TypeWrapper,
+				outTypeWrapper, new CoGroupedWindowGroupReduceInvokable<IN1, IN2, OUT>(coReducer,
+						windowSize1, windowSize2, slideInterval1, slideInterval2, keyPosition1,
+						keyPosition2, timestamp1, timestamp2));
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ad983377/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java
index 334caa7..07e3d6c 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java
@@ -21,8 +21,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
 
 public class CoBatchGroupReduceInvokable<IN1, IN2, OUT> extends CoGroupReduceInvokable<IN1, IN2, OUT> {
-
 	private static final long serialVersionUID = 1L;
+	
 	protected long startCounter1;
 	protected long startCounter2;
 	protected long endCounter1;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ad983377/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
index 1a6d678..23dcb57 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
@@ -17,8 +17,7 @@
 
 package org.apache.flink.streaming.state;
 
-import java.util.ArrayDeque;
-import java.util.Deque;
+import java.io.Serializable;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Queue;
@@ -28,14 +27,16 @@ import java.util.Queue;
  * queue if full and a new element is added, the elements that belong to the
  * first sliding interval are removed.
  */
-public class CircularFifoList<T> {
+public class CircularFifoList<T> implements Serializable {
+	private static final long serialVersionUID = 1L;
+
 	private Queue<T> queue;
-	private Deque<Long> slideSizes;
+	private Queue<Long> slideSizes;
 	private long counter;
 
 	public CircularFifoList() {
 		this.queue = new LinkedList<T>();
-		this.slideSizes = new ArrayDeque<Long>();
+		this.slideSizes = new LinkedList<Long>();
 		this.counter = 0;
 	}
 
@@ -50,10 +51,10 @@ public class CircularFifoList<T> {
 	}
 
 	public void shiftWindow() {
-		for (int i = 0; i < slideSizes.getFirst(); i++) {
+		Long firstSlideSize = slideSizes.remove();
+		for (int i = 0; i < firstSlideSize; i++) {
 			queue.remove();
 		}
-		slideSizes.remove();
 	}
 
 	public Iterator<T> getIterator() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ad983377/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.java
index 7a7a07a..ed1cd80 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.state;
 
+import java.io.Serializable;
 import java.util.Iterator;
 
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -25,7 +26,9 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
  * Simple wrapper class to convert an Iterator<StreamRecord<T>> to an
  * Iterator<T> iterator by invoking the getObject() method on every element.
  */
-public class StreamIterator<T> implements Iterator<T> {
+public class StreamIterator<T> implements Iterator<T>, Serializable {
+	private static final long serialVersionUID = 1L;
+
 	private Iterator<StreamRecord<T>> iterator = null;
 
 	public void load(Iterator<StreamRecord<T>> iterator) {


[03/12] [FLINK-1102] [streaming] Projection operator added to DataStream

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index c3231d7..e7a68d3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -35,7 +35,7 @@ import org.apache.flink.streaming.api.function.source.SourceFunction;
 import org.apache.flink.streaming.api.invokable.SourceInvokable;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
 
 /**
  * {@link ExecutionEnvironment} for streaming jobs. An instance of it is
@@ -224,7 +224,7 @@ public abstract class StreamExecutionEnvironment {
 					"fromElements needs at least one element as argument");
 		}
 
-		TypeSerializerWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data[0]);
+		TypeWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data[0]);
 		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements",
 				outTypeWrapper);
 
@@ -259,7 +259,7 @@ public abstract class StreamExecutionEnvironment {
 			throw new IllegalArgumentException("Collection must not be empty");
 		}
 
-		TypeSerializerWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data.iterator()
+		TypeWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data.iterator()
 				.next());
 		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements",
 				outTypeWrapper);
@@ -305,7 +305,7 @@ public abstract class StreamExecutionEnvironment {
 	 * @return the data stream constructed
 	 */
 	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, int parallelism) {
-		TypeSerializerWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(function,
+		TypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(function,
 				SourceFunction.class, 0);
 		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "source",
 				outTypeWrapper);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
index b3cd57f..476c519 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
@@ -43,7 +43,7 @@ public abstract class StreamInvokable<IN, OUT> implements Serializable {
 	private static final Logger LOG = LoggerFactory.getLogger(StreamInvokable.class);
 
 	protected MutableObjectIterator<StreamRecord<IN>> recordIterator;
-	protected StreamRecordSerializer<IN> serializer;
+	protected StreamRecordSerializer<IN> inSerializer;
 	protected StreamRecord<IN> reuse;
 	protected boolean isMutable;
 
@@ -73,8 +73,8 @@ public abstract class StreamInvokable<IN, OUT> implements Serializable {
 			StreamRecordSerializer<IN> serializer, boolean isMutable) {
 		this.collector = collector;
 		this.recordIterator = recordIterator;
-		this.serializer = serializer;
-		if(this.serializer != null){
+		this.inSerializer = serializer;
+		if(this.inSerializer != null){
 			this.reuse = serializer.createInstance();
 		}
 		this.isMutable = isMutable;
@@ -84,7 +84,7 @@ public abstract class StreamInvokable<IN, OUT> implements Serializable {
 	 * Re-initializes the object in which the next input record will be read in
 	 */
 	protected void resetReuse() {
-		this.reuse = serializer.createInstance();
+		this.reuse = inSerializer.createInstance();
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
index f306dac..1cfc2d2 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
@@ -117,7 +117,7 @@ public class BatchReduceInvokable<OUT> extends StreamInvokable<OUT, OUT> {
 	@Override
 	public void open(Configuration config) throws Exception {
 		super.open(config);
-		this.typeSerializer = serializer.getObjectSerializer();
+		this.typeSerializer = inSerializer.getObjectSerializer();
 	}
 
 	protected class StreamBatch implements Serializable {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java
new file mode 100644
index 0000000..f72f66e
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java
@@ -0,0 +1,65 @@
+/**
+ * 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.invokable.operator;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.invokable.StreamInvokable;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
+
+public class ProjectInvokable<IN, OUT extends Tuple> extends StreamInvokable<IN, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	transient OUT outTuple;
+	TypeWrapper<OUT> outTypeWrapper;
+	int[] fields;
+	int numFields;
+
+	public ProjectInvokable(int[] fields, TypeWrapper<OUT> outTypeWrapper) {
+		super(null);
+		this.fields = fields;
+		this.numFields = this.fields.length;
+		this.outTypeWrapper = outTypeWrapper;
+	}
+
+	@Override
+	protected void immutableInvoke() throws Exception {
+		mutableInvoke();
+	}
+
+	@Override
+	protected void mutableInvoke() throws Exception {
+		while ((reuse = recordIterator.next(reuse)) != null) {
+			callUserFunctionAndLogException();
+		}
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+		for (int i = 0; i < this.numFields; i++) {
+			outTuple.setField(reuse.getField(fields[i]), i);
+		}
+		collector.collect(outTuple);
+	}
+
+	@Override
+	public void open(Configuration config) throws Exception {
+		super.open(config);
+		outTuple = outTypeWrapper.getTypeInfo().createSerializer().createInstance();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
index ea1a750..4255912 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/FunctionTypeWrapper.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 
-public class FunctionTypeWrapper<T> extends TypeSerializerWrapper<T> {
+public class FunctionTypeWrapper<T> extends TypeWrapper<T> {
 	private static final long serialVersionUID = 1L;
 
 	private Function function;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
index 487c0a3..6bf90c4 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ObjectTypeWrapper.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 public class ObjectTypeWrapper<T> extends
-		TypeSerializerWrapper<T> {
+		TypeWrapper<T> {
 	private static final long serialVersionUID = 1L;
 
 	private T instance;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ProjectTypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ProjectTypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ProjectTypeWrapper.java
new file mode 100644
index 0000000..9e8d4b4
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/ProjectTypeWrapper.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.util.serialization;
+
+import java.io.IOException;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+public class ProjectTypeWrapper<IN,OUT extends Tuple> extends
+		TypeWrapper<OUT> {
+	private static final long serialVersionUID = 1L;
+
+
+	private TypeWrapper<IN> inType;
+	Class<?>[] givenTypes;
+	int[] fields;
+
+	public ProjectTypeWrapper(TypeWrapper<IN> inType,int[] fields,Class<?>[] givenTypes) {
+		this.inType = inType;
+		this.givenTypes = givenTypes;
+		this.fields = fields;
+		setTypeInfo();
+	}
+
+	private void readObject(java.io.ObjectInputStream in) throws IOException,
+			ClassNotFoundException {
+		in.defaultReadObject();
+		setTypeInfo();
+	}
+
+	@Override
+	protected void setTypeInfo() {
+		TypeInformation<?>[] outTypes = extractFieldTypes();
+		this.typeInfo = new TupleTypeInfo<OUT>(outTypes);
+	}
+	
+	private TypeInformation<?>[] extractFieldTypes() {
+		
+		TupleTypeInfo<?> inTupleType = (TupleTypeInfo<?>) inType.getTypeInfo();
+		TypeInformation<?>[] fieldTypes = new TypeInformation[fields.length];
+				
+		for(int i=0; i<fields.length; i++) {
+			
+			if(inTupleType.getTypeAt(fields[i]).getTypeClass() != givenTypes[i]) {
+				throw new IllegalArgumentException("Given types do not match types of input data set.");
+			}
+				
+			fieldTypes[i] = inTupleType.getTypeAt(fields[i]);
+		}
+		
+		return fieldTypes;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
deleted file mode 100644
index ca7a9d2..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeSerializerWrapper.java
+++ /dev/null
@@ -1,38 +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.util.serialization;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-public abstract class TypeSerializerWrapper<T>
-		implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	protected transient TypeInformation<T> typeInfo = null;
-	
-	public TypeInformation<T> getTypeInfo() {
-		if (typeInfo == null) {
-			throw new RuntimeException("There is no TypeInformation in the wrapper");
-		}
-		return typeInfo;
-	}
-
-	protected abstract void setTypeInfo();
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeWrapper.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeWrapper.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeWrapper.java
new file mode 100644
index 0000000..a2e16b6
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeWrapper.java
@@ -0,0 +1,38 @@
+/*
+ * 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.serialization;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+public abstract class TypeWrapper<T>
+		implements Serializable {
+	private static final long serialVersionUID = 1L;
+
+	protected transient TypeInformation<T> typeInfo = null;
+	
+	public TypeInformation<T> getTypeInfo() {
+		if (typeInfo == null) {
+			throw new RuntimeException("There is no TypeInformation in the wrapper");
+		}
+		return typeInfo;
+	}
+
+	protected abstract void setTypeInfo();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/ProjectTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/ProjectTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/ProjectTest.java
new file mode 100644
index 0000000..5157dcb
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/ProjectTest.java
@@ -0,0 +1,66 @@
+/**
+ * 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.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.apache.flink.streaming.util.serialization.ObjectTypeWrapper;
+import org.apache.flink.streaming.util.serialization.ProjectTypeWrapper;
+import org.apache.flink.streaming.util.serialization.TypeWrapper;
+import org.junit.Test;
+
+public class ProjectTest implements Serializable {
+	private static final long serialVersionUID = 1L;
+
+	@Test
+	public void test() {
+
+		TypeWrapper<Tuple5<Integer, String, Integer, String, Integer>> inTypeWrapper = new ObjectTypeWrapper<Tuple5<Integer, String, Integer, String, Integer>>(
+				new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "b", 4));
+
+		int[] fields = new int[] { 4, 4, 3 };
+		Class<?>[] classes = new Class<?>[] { Integer.class, Integer.class, String.class };
+
+		TypeWrapper<Tuple3<Integer, Integer, String>> outTypeWrapper = new ProjectTypeWrapper<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>>(
+				inTypeWrapper, fields, classes);
+
+		ProjectInvokable<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>> invokable = new ProjectInvokable<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>>(
+				fields, outTypeWrapper);
+
+		List<Tuple5<Integer, String, Integer, String, Integer>> input = new ArrayList<Tuple5<Integer, String, Integer, String, Integer>>();
+		input.add(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "b", 4));
+		input.add(new Tuple5<Integer, String, Integer, String, Integer>(2, "s", 3, "c", 2));
+		input.add(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "c", 2));
+		input.add(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "a", 7));
+
+		List<Tuple3<Integer, Integer, String>> expected = new ArrayList<Tuple3<Integer, Integer, String>>();
+		expected.add(new Tuple3<Integer, Integer, String>(4, 4, "b"));
+		expected.add(new Tuple3<Integer, Integer, String>(2, 2, "c"));
+		expected.add(new Tuple3<Integer, Integer, String>(2, 2, "c"));
+		expected.add(new Tuple3<Integer, Integer, String>(7, 7, "a"));
+
+		assertEquals(expected, MockInvokable.createAndExecute(invokable, input));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
index f470c76..0119f04 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamvertex/StreamVertexTest.java
@@ -116,6 +116,11 @@ public class StreamVertexTest {
 			fail();
 		} catch (IllegalArgumentException e) {
 		}
+		try {
+			env.generateSequence(1, 10).project(2);
+			fail();
+		} catch (RuntimeException e) {
+		}
 
 		try {
 			env.readTextFile("random/path/that/is/not/valid");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCollector.java
index c4682dd..e8b96c5 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCollector.java
@@ -17,20 +17,24 @@
 
 package org.apache.flink.streaming.util;
 
+import java.io.Serializable;
 import java.util.Collection;
 
+import org.apache.commons.lang3.SerializationUtils;
 import org.apache.flink.util.Collector;
 
 public class MockCollector<T> implements Collector<T> {
 	private Collection<T> outputs;
-	
+
 	public MockCollector(Collection<T> outputs) {
 		this.outputs = outputs;
 	}
 
 	@Override
 	public void collect(T record) {
-		outputs.add(record);
+		T copied = SerializationUtils.deserialize(SerializationUtils
+				.serialize((Serializable) record));
+		outputs.add(copied);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4175dca8/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
index d9cde4f..8795126 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/serialization/TypeSerializationTest.java
@@ -38,12 +38,12 @@ public class TypeSerializationTest {
 	@SuppressWarnings("unchecked")
 	@Test
 	public void functionTypeSerializationTest() {
-		TypeSerializerWrapper<Integer> ser = new FunctionTypeWrapper<Integer>(new MyMap(),
+		TypeWrapper<Integer> ser = new FunctionTypeWrapper<Integer>(new MyMap(),
 				RichMapFunction.class, 0);
 
 		byte[] serializedType = SerializationUtils.serialize(ser);
 
-		TypeSerializerWrapper<Integer> ser2 = (TypeSerializerWrapper<Integer>) SerializationUtils
+		TypeWrapper<Integer> ser2 = (TypeWrapper<Integer>) SerializationUtils
 				.deserialize(serializedType);
 
 		assertNotNull(ser.getTypeInfo());
@@ -57,11 +57,11 @@ public class TypeSerializationTest {
 	public void objectTypeSerializationTest() {
 		Integer instance = new Integer(22);
 		
-		TypeSerializerWrapper<Integer> ser = new ObjectTypeWrapper<Integer>(instance);
+		TypeWrapper<Integer> ser = new ObjectTypeWrapper<Integer>(instance);
 		
 		byte[] serializedType = SerializationUtils.serialize(ser);
 
-		TypeSerializerWrapper<Integer> ser2 = (TypeSerializerWrapper<Integer>) SerializationUtils
+		TypeWrapper<Integer> ser2 = (TypeWrapper<Integer>) SerializationUtils
 				.deserialize(serializedType);
 
 		assertNotNull(ser.getTypeInfo());