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/20 15:10:44 UTC

[01/18] git commit: [streaming] Fixed SlidingWindowStateIterator

Repository: incubator-flink
Updated Branches:
  refs/heads/master 1c58eb8b0 -> 439ca7ffe


[streaming] Fixed SlidingWindowStateIterator


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

Branch: refs/heads/master
Commit: d97efdee43f3cad28a20280870f1abeee0dd1bd5
Parents: 1c58eb8
Author: ghermann <re...@gmail.com>
Authored: Fri Aug 29 17:18:12 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:03 2014 +0200

----------------------------------------------------------------------
 .../operator/BatchReduceInvokable.java          | 46 +++++++++++------
 .../operator/WindowReduceInvokable.java         | 52 +++-----------------
 .../state/SlidingWindowStateIterator.java       | 24 +++++++--
 .../operator/WindowReduceInvokableTest.java     | 26 ++++++++--
 .../streaming/state/SlidingWindowStateTest.java | 50 ++++++++++++++++++-
 5 files changed, 129 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d97efdee/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 acd9f62..0ec94ca 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
@@ -38,7 +38,7 @@ public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, O
 	protected long granularity;
 	protected int listSize;
 	protected transient SlidingWindowState<IN> state;
-	
+
 	private long batchSize;
 	private int counter = 0;
 
@@ -59,15 +59,17 @@ public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, O
 
 	@Override
 	protected void immutableInvoke() throws Exception {
-		if ((reuse = recordIterator.next(reuse)) == null) {
+		if (getNextRecord() == null) {
 			throw new RuntimeException("DataStream must not be empty");
 		}
-	
+
+		initializeAtFirstRecord();
+
 		while (reuse != null && !state.isFull()) {
 			collectOneUnit();
 		}
 		reduce();
-	
+
 		while (reuse != null) {
 			for (int i = 0; i < slideSize / granularity; i++) {
 				if (reuse != null) {
@@ -78,20 +80,35 @@ public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, O
 		}
 	}
 
+	protected void initializeAtFirstRecord() {
+		counter = 0;
+	}
+
 	protected void collectOneUnit() throws IOException {
 		ArrayList<StreamRecord<IN>> list;
-		list = new ArrayList<StreamRecord<IN>>(listSize);
-	
-		do {
-			list.add(reuse);
-			resetReuse();
-		} while ((reuse = recordIterator.next(reuse)) != null && batchNotFull());
+
+		if (!batchNotFull()) {
+			list = new ArrayList<StreamRecord<IN>>();
+		} else {
+			list = new ArrayList<StreamRecord<IN>>(listSize);
+
+			do {
+				list.add(reuse);
+				resetReuse();
+			} while (getNextRecord() != null && batchNotFull());
+		}
 		state.pushBack(list);
 	}
 
-	
+	protected StreamRecord<IN> getNextRecord() throws IOException {
+		reuse = recordIterator.next(reuse);
+		if (reuse != null) {
+			counter++;
+		}
+		return reuse;
+	}
+
 	protected boolean batchNotFull() {
-		counter++;
 		if (counter < granularity) {
 			return true;
 		} else {
@@ -104,12 +121,12 @@ public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, O
 		userIterator = state.getIterator();
 		callUserFunctionAndLogException();
 	}
-	
+
 	@Override
 	protected void callUserFunction() throws Exception {
 		reducer.reduce(userIterable, collector);
 	}
-	
+
 	@Override
 	public void open(Configuration parameters) throws Exception {
 		super.open(parameters);
@@ -126,5 +143,4 @@ public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, O
 
 	}
 
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d97efdee/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 76d1768..49e2411 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -18,7 +18,6 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import java.io.IOException;
-import java.util.ArrayList;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.configuration.Configuration;
@@ -38,55 +37,20 @@ public class WindowReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN, OUT
 	}
 
 	@Override
-	protected void immutableInvoke() throws Exception {
-		if ((reuse = recordIterator.next(reuse)) == null) {
-			throw new RuntimeException("DataStream must not be empty");
-		}
-
-		nextRecordTime = timestamp.getTimestamp(reuse.getObject()); // **
-		startTime = nextRecordTime - (nextRecordTime % granularity); // **
-
-		while (reuse != null && !state.isFull()) {
-			collectOneUnit();
-		}
-		reduce();
-
-		while (reuse != null) {
-			for (int i = 0; i < slideSize / granularity; i++) {
-				if (reuse != null) {
-					collectOneUnit();
-				}
-			}
-			reduce();
-		}
+	protected void initializeAtFirstRecord() {
+		startTime = nextRecordTime - (nextRecordTime % granularity);
 	}
 
-	@Override
-	protected void collectOneUnit() throws IOException {
-		ArrayList<StreamRecord<IN>> list;
-		if (nextRecordTime > startTime + granularity - 1) {
-			list = new ArrayList<StreamRecord<IN>>();
-			startTime += granularity;
-		} else {
-			list = new ArrayList<StreamRecord<IN>>(listSize);
-
-			list.add(reuse);
-			resetReuse();
-
-			while ((reuse = recordIterator.next(reuse)) != null && batchNotFull()) {
-				list.add(reuse);
-				resetReuse();
-			}
+	protected StreamRecord<IN> getNextRecord() throws IOException {
+		reuse = recordIterator.next(reuse);
+		if (reuse != null) {
+			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
 		}
-		state.pushBack(list);
-//		System.out.println(list);
-//		System.out.println(startTime + " - " + (startTime + granularity - 1) + " ("
-//				+ nextRecordTime + ")");
+		return reuse;
 	}
-
+	
 	@Override
 	protected boolean batchNotFull() {
-		nextRecordTime = timestamp.getTimestamp(reuse.getObject());
 		if (nextRecordTime < startTime + granularity) {
 			return true;
 		} else {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d97efdee/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
index 13d9cc8..1a88095 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.state;
 
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 import org.apache.commons.collections.buffer.CircularFifoBuffer;
 import org.apache.flink.streaming.api.invokable.operator.BatchIterator;
@@ -27,6 +28,8 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
 
 	private CircularFifoBuffer buffer;
+	// private StreamRecord<T> nextElement;
+
 	private Iterator<Collection<StreamRecord<T>>> iterator;
 	private Iterator<StreamRecord<T>> subIterator;
 	private Iterator<StreamRecord<T>> streamRecordIterator;
@@ -37,16 +40,29 @@ public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
 	}
 
 	public boolean hasNext() {
+		while (iterator.hasNext() && !subIterator.hasNext()) {
+			subIterator = iterator.next().iterator();
+		}
+
 		return subIterator.hasNext();
 	}
 
 	public T next() {
-		T nextElement = subIterator.next().getObject();
-		if (!subIterator.hasNext()) {
-			if (iterator.hasNext()) {
-				subIterator = iterator.next().iterator();
+		T nextElement;
+
+		if (hasNext()) {
+			nextElement = subIterator.next().getObject();
+
+			if (!subIterator.hasNext()) {
+				if (iterator.hasNext()) {
+					subIterator = iterator.next().iterator();
+				}
 			}
+		} else {
+			throw new NoSuchElementException("There is no more element in the current batch");
 		}
+
+
 		return nextElement;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d97efdee/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
index 3dbabbb..6dec400 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
@@ -80,7 +80,18 @@ public class WindowReduceInvokableTest {
 		windowSize = 10;
 		slideSize = 5;
 		timestamps = Arrays.asList(101L, 103L, 121L, 122L, 123L, 124L, 180L, 181L, 185L, 190L);
-		expectedResults.add(Arrays.asList("1", "2", EOW, EOW, EOW, EOW, "3", "4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", "8", "9", EOW, "9", "10", EOW));
+		expectedResults.add(Arrays.asList("1", "2", EOW, EOW, EOW, "3", "4", "5", "6", EOW, "3",
+				"4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", "8",
+				EOW, "7", "8", "9", EOW, "9", "10", EOW));
+		invokables.add(new WindowReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
+				windowSize, slideSize, new MyTimestamp(timestamps)));
+
+		windowSize = 10;
+		slideSize = 4;
+		timestamps = Arrays.asList(101L, 103L, 110L, 112L, 113L, 114L, 120L, 121L, 125L, 130L);
+		expectedResults.add(Arrays.asList("1", "2", EOW, "3", "4", "5", EOW, "3", "4", "5", "6",
+				EOW, "4", "5", "6", "7", "8", EOW, "7", "8", "9", EOW, "7", "8", "9", EOW, "9",
+				"10", EOW));
 		invokables.add(new WindowReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
 				windowSize, slideSize, new MyTimestamp(timestamps)));
 	}
@@ -88,13 +99,18 @@ public class WindowReduceInvokableTest {
 	@Test
 	public void slidingBatchReduceTest() {
 		List<List<String>> actualResults = new ArrayList<List<String>>();
-		
+
 		for (WindowReduceInvokable<Integer, String> invokable : invokables) {
-			actualResults.add(MockInvokable.createAndExecute(invokable,
-					Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)));
+			List<String> result = MockInvokable.createAndExecute(invokable,
+					Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+			actualResults.add(result);
 		}
 
-		assertEquals(expectedResults, actualResults);
+		Iterator<List<String>> actualResult = actualResults.iterator();
+
+		for (List<String> expectedResult : expectedResults) {
+			assertEquals(expectedResult, actualResult.next());
+		}
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d97efdee/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
index 081fdf1..49e939e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/state/SlidingWindowStateTest.java
@@ -20,7 +20,9 @@ package org.apache.flink.streaming.state;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -35,7 +37,7 @@ public class SlidingWindowStateTest {
 
 	@SuppressWarnings("unchecked")
 	@Test
-	public void test() {
+	public void basicSlidingWindowStateTest() {
 		SlidingWindowState<Integer> state = new SlidingWindowState<Integer>(SLIDING_BATCH_SIZE,
 				SLIDE_SIZE, UNIT);
 		state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(0)));
@@ -65,6 +67,52 @@ public class SlidingWindowStateTest {
 		}
 		assertEquals(getExpectedSet(2, 4), actualSet);
 	}
+	
+	private final static int WINDOW_SIZE = 10;
+	private final static int WINDOW_SLIDE_SIZE = 4;
+	private static final int WINDOW_UNIT = 2;
+	
+	@SuppressWarnings("unchecked")
+	@Test
+	public void slidingWithGreaterUnit() {
+		SlidingWindowState<Integer> state = new SlidingWindowState<Integer>(WINDOW_SIZE,
+				WINDOW_SLIDE_SIZE, WINDOW_UNIT);
+		state.pushBack(new ArrayList<StreamRecord<Integer>>());
+		state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(1)));
+		state.pushBack(new ArrayList<StreamRecord<Integer>>());
+		state.pushBack(Arrays.asList(new StreamRecord<Integer>().setObject(2), new StreamRecord<Integer>().setObject(3)));
+		state.pushBack(new ArrayList<StreamRecord<Integer>>());
+
+		SortedSet<Integer> actualSet = new TreeSet<Integer>();
+		SlidingWindowStateIterator<Integer> iterator = state.getIterator();
+		
+		iterator.hasNext();
+		iterator.hasNext();
+		while (iterator.hasNext()) {
+			iterator.hasNext();
+			iterator.hasNext();
+			actualSet.add(iterator.next());
+			iterator.hasNext();
+			iterator.hasNext();
+		}
+		
+		assertEquals(getExpectedSet(1, 3), actualSet);
+		actualSet.clear();
+		
+		Iterator<StreamRecord<Integer>> streamRecordIterator = state.getStreamRecordIterator();
+		
+		streamRecordIterator.hasNext();
+		streamRecordIterator.hasNext();
+		while (streamRecordIterator.hasNext()) {
+			streamRecordIterator.hasNext();
+			streamRecordIterator.hasNext();
+			actualSet.add(streamRecordIterator.next().getObject());
+			streamRecordIterator.hasNext();
+			streamRecordIterator.hasNext();
+		}
+		
+		assertEquals(getExpectedSet(1, 3), actualSet);
+	}
 
 	private SortedSet<Integer> getExpectedSet(int from, int to) {
 		SortedSet<Integer> expectedSet = new TreeSet<Integer>();


[16/18] git commit: [streaming] Added CoBatchGroupReduceInvokable, CoWindowGroupReduceInvokable and grouped variants

Posted by mb...@apache.org.
[streaming] Added CoBatchGroupReduceInvokable, CoWindowGroupReduceInvokable and grouped variants


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

Branch: refs/heads/master
Commit: b6ffdbad1e6779d8c02e1d0ac77ae6da4ad4a216
Parents: d0dd513
Author: szape <ne...@gmail.com>
Authored: Wed Sep 10 10:41:28 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 .../api/datastream/ConnectedDataStream.java     |   4 +-
 .../datastream/GroupedConnectedDataStream.java  |   4 +-
 .../api/datastream/GroupedDataStream.java       |   6 +-
 .../operator/GroupReduceInvokable.java          |  56 ----
 .../operator/GroupedReduceInvokable.java        |  56 ++++
 .../co/CoBatchGroupReduceInvokable.java         | 117 +++++++++
 .../operator/co/CoGroupReduceInvokable.java     | 140 ++++++----
 .../co/CoGroupedBatchGroupReduceInvokable.java  |  99 +++++++
 .../operator/co/CoGroupedReduceInvokable.java   |  85 ++++++
 .../co/CoGroupedWindowGroupReduceInvokable.java | 102 ++++++++
 .../api/invokable/operator/co/CoInvokable.java  |  14 +-
 .../operator/co/CoReduceInvokable.java          |  70 -----
 .../operator/co/CoStreamReduceInvokable.java    |  70 +++++
 .../co/CoWindowGroupReduceInvokable.java        |  98 +++++++
 .../flink/streaming/state/CircularFifoList.java |  67 +++++
 .../flink/streaming/state/StreamIterator.java   |  55 ++++
 .../streaming/api/AggregationFunctionTest.java  |   8 +-
 .../operator/CoBatchGroupReduceTest.java        |  96 +++++++
 .../api/invokable/operator/CoFlatMapTest.java   |   4 +-
 .../invokable/operator/CoGroupReduceTest.java   |  96 -------
 .../operator/CoGroupedBatchGroupReduceTest.java | 116 +++++++++
 .../invokable/operator/CoGroupedReduceTest.java |  94 +++++++
 .../CoGroupedWindowGroupReduceTest.java         | 223 ++++++++++++++++
 .../api/invokable/operator/CoReduceTest.java    |  71 -----
 .../invokable/operator/CoStreamReduceTest.java  |  71 +++++
 .../operator/CoWindowGroupReduceTest.java       | 256 +++++++++++++++++++
 .../operator/GroupReduceInvokableTest.java      |  54 ----
 .../operator/GroupedReduceInvokableTest.java    |  54 ++++
 28 files changed, 1782 insertions(+), 404 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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 e00919f..9529dcd 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
@@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
 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.CoReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoStreamReduceInvokable;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
 import org.apache.flink.types.TypeInformation;
@@ -196,7 +196,7 @@ public class ConnectedDataStream<IN1, IN2> {
 				CoReduceFunction.class, 2);
 
 		return addCoFunction("coReduce", coReducer, in1TypeWrapper, in2TypeWrapper, outTypeWrapper,
-				new CoReduceInvokable<IN1, IN2, OUT>(coReducer));
+				new CoStreamReduceInvokable<IN1, IN2, OUT>(coReducer));
 	}
 
 	protected <OUT> SingleOutputStreamOperator<OUT, ?> addCoFunction(String functionName,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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 8276561..8a16b98 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
@@ -20,7 +20,7 @@ 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.CoReduceFunction;
-import org.apache.flink.streaming.api.invokable.operator.co.CoGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedReduceInvokable;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 
 public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN1, IN2> {
@@ -61,7 +61,7 @@ public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN
 				CoReduceFunction.class, 2);
 
 		return addCoFunction("coReduce", coReducer, in1TypeWrapper, in2TypeWrapper, outTypeWrapper,
-				new CoGroupReduceInvokable<IN1, IN2, OUT>(coReducer, keyPosition1, keyPosition2));
+				new CoGroupedReduceInvokable<IN1, IN2, OUT>(coReducer, keyPosition1, keyPosition2));
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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 138a6f8..30826d3 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
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupedWindowGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
 import org.apache.flink.streaming.api.invokable.util.TimeStamp;
@@ -69,7 +69,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	public SingleOutputStreamOperator<OUT, ?> reduce(ReduceFunction<OUT> reducer) {
 		return addFunction("groupReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				ReduceFunction.class, 0), new FunctionTypeWrapper<OUT>(reducer,
-				ReduceFunction.class, 0), new GroupReduceInvokable<OUT>(reducer, keyPosition));
+				ReduceFunction.class, 0), new GroupedReduceInvokable<OUT>(reducer, keyPosition));
 	}
 
 	/**
@@ -240,7 +240,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	@Override
 	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
 
-		GroupReduceInvokable<OUT> invokable = new GroupReduceInvokable<OUT>(aggregate, keyPosition);
+		GroupedReduceInvokable<OUT> invokable = new GroupedReduceInvokable<OUT>(aggregate, keyPosition);
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("groupReduce", aggregate,
 				null, null, invokable);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
deleted file mode 100755
index 73523d9..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokable.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.invokable.operator;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.state.MutableTableState;
-
-public class GroupReduceInvokable<IN> extends StreamReduceInvokable<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private int keyPosition;
-	private MutableTableState<Object, IN> values;
-	private IN reduced;
-
-	public GroupReduceInvokable(ReduceFunction<IN> reducer, int keyPosition) {
-		super(reducer);
-		this.keyPosition = keyPosition;
-		values = new MutableTableState<Object, IN>();
-	}
-
-	@Override
-	protected void reduce() throws Exception {
-		Object key = reuse.getField(keyPosition);
-		currentValue = values.get(key);
-		nextValue = reuse.getObject();
-		if (currentValue != null) {
-			callUserFunctionAndLogException();
-			values.put(key, reduced);
-			collector.collect(reduced);
-		} else {
-			values.put(key, nextValue);
-			collector.collect(nextValue);
-		}
-	}
-
-	@Override
-	protected void callUserFunction() throws Exception {
-		reduced = reducer.reduce(currentValue, nextValue);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokable.java
new file mode 100755
index 0000000..6423492
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokable.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class GroupedReduceInvokable<IN> extends StreamReduceInvokable<IN> {
+	private static final long serialVersionUID = 1L;
+
+	private int keyPosition;
+	private MutableTableState<Object, IN> values;
+	private IN reduced;
+
+	public GroupedReduceInvokable(ReduceFunction<IN> reducer, int keyPosition) {
+		super(reducer);
+		this.keyPosition = keyPosition;
+		values = new MutableTableState<Object, IN>();
+	}
+
+	@Override
+	protected void reduce() throws Exception {
+		Object key = reuse.getField(keyPosition);
+		currentValue = values.get(key);
+		nextValue = reuse.getObject();
+		if (currentValue != null) {
+			callUserFunctionAndLogException();
+			values.put(key, reduced);
+			collector.collect(reduced);
+		} else {
+			values.put(key, nextValue);
+			collector.collect(nextValue);
+		}
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+		reduced = reducer.reduce(currentValue, nextValue);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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
new file mode 100644
index 0000000..1cdf8c7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchGroupReduceInvokable.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator.co;
+
+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;
+	protected long endCounter2;
+
+	public CoBatchGroupReduceInvokable(CoGroupReduceFunction<IN1, IN2, OUT> reduceFunction,
+			long windowSize1, long windowSize2, long slideInterval1, long slideInterval2) {
+		super(reduceFunction, windowSize1, windowSize2, slideInterval1, slideInterval2);
+	}
+
+	@Override
+	protected void handleStream1() throws Exception {
+		circularList1.add(reuse1);
+		if (windowStart1()) {
+			circularList1.newSlide();
+		}
+		if (windowEnd1()) {
+			reduce1();
+			circularList1.shiftWindow();
+		}
+	}
+
+	@Override
+	protected void handleStream2() throws Exception {
+		circularList2.add(reuse2);
+		if (windowStart2()) {
+			circularList2.newSlide();
+		}
+		if (windowEnd2()) {
+			reduce2();
+			circularList2.shiftWindow();
+		}
+	}
+
+	@Override
+	protected boolean windowStart1() throws Exception {
+		if (startCounter1 == slideInterval1) {
+			startCounter1 = 0;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowStart2() throws Exception {
+		if (startCounter2 == slideInterval2) {
+			startCounter2 = 0;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowEnd1() throws Exception {
+		if (endCounter1 == windowSize1) {
+			endCounter1 -= slideInterval1;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowEnd2() throws Exception {
+		if (endCounter2 == windowSize2) {
+			endCounter2 -= slideInterval2;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		startCounter1 = 0;
+		startCounter2 = 0;
+		endCounter1 = 0;
+		endCounter2 = 0;
+	}
+
+	@Override
+	protected void initialize1() {
+		startCounter1++;
+		endCounter1++;
+	}
+
+	@Override
+	protected void initialize2() {
+		startCounter2++;
+		endCounter2++;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
index ab827d8..9700c70 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
@@ -17,69 +17,123 @@
 
 package org.apache.flink.streaming.api.invokable.operator.co;
 
-import org.apache.flink.streaming.api.function.co.CoReduceFunction;
-import org.apache.flink.streaming.state.MutableTableState;
+import java.util.Iterator;
 
-public class CoGroupReduceInvokable<IN1, IN2, OUT> extends CoReduceInvokable<IN1, IN2, OUT> {
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.CircularFifoList;
+import org.apache.flink.streaming.state.StreamIterator;
+
+public abstract class CoGroupReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
 	private static final long serialVersionUID = 1L;
 
-	private int keyPosition1;
-	private int keyPosition2;
-	private MutableTableState<Object, IN1> values1;
-	private MutableTableState<Object, IN2> values2;
-	IN1 reduced1;
-	IN2 reduced2;
-
-	public CoGroupReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer, int keyPosition1,
-			int keyPosition2) {
-		super(coReducer);
-		this.coReducer = coReducer;
-		this.keyPosition1 = keyPosition1;
-		this.keyPosition2 = keyPosition2;
-		values1 = new MutableTableState<Object, IN1>();
-		values2 = new MutableTableState<Object, IN2>();
+	protected CoGroupReduceFunction<IN1, IN2, OUT> coReducer;
+	protected StreamIterator<IN1> userIterator1;
+	protected StreamIterator<IN2> userIterator2;
+	protected Iterable<IN1> userIterable1;
+	protected Iterable<IN2> userIterable2;
+	protected long windowSize1;
+	protected long windowSize2;
+	protected long slideInterval1;
+	protected long slideInterval2;
+	protected CircularFifoList<StreamRecord<IN1>> circularList1;
+	protected CircularFifoList<StreamRecord<IN2>> circularList2;
+	protected long WindowStartTime1;
+	protected long WindowStartTime2;
+	protected long WindowEndTime1;
+	protected long WindowEndTime2;
+
+	public CoGroupReduceInvokable(CoGroupReduceFunction<IN1, IN2, OUT> reduceFunction,
+			long windowSize1, long windowSize2, long slideInterval1, long slideInterval2) {
+		super(reduceFunction);
+		this.coReducer = reduceFunction;
+		this.userIterator1 = new StreamIterator<IN1>();
+		this.userIterator2 = new StreamIterator<IN2>();
+		this.windowSize1 = windowSize1;
+		this.windowSize2 = windowSize2;
+		this.slideInterval1 = slideInterval1;
+		this.slideInterval2 = slideInterval2;
+		this.circularList1 = new CircularFifoList<StreamRecord<IN1>>();
+		this.circularList2 = new CircularFifoList<StreamRecord<IN2>>();
+	}
+
+	@Override
+	protected void mutableInvoke() throws Exception {
+		throw new RuntimeException("Reducing mutable sliding batch is not supported.");
 	}
 
 	@Override
-	public void handleStream1() throws Exception {
-		Object key = reuse1.getField(keyPosition1);
-		currentValue1 = values1.get(key);
-		nextValue1 = reuse1.getObject();
-		if (currentValue1 != null) {
-			callUserFunctionAndLogException1();
-			values1.put(key, reduced1);
-			collector.collect(coReducer.map1(reduced1));
-		} else {
-			values1.put(key, nextValue1);
-			collector.collect(coReducer.map1(nextValue1));
+	protected void handleStream1() throws Exception {
+		while (windowStart1()) {
+			circularList1.newSlide();
+		}
+		while (windowEnd1()) {
+			reduce1();
+			circularList1.shiftWindow();
 		}
+		circularList1.add(reuse1);
 	}
 
 	@Override
-	public void handleStream2() throws Exception {
-		Object key = reuse2.getField(keyPosition2);
-		currentValue2 = values2.get(key);
-		nextValue2 = reuse2.getObject();
-		if (currentValue2 != null) {
-			callUserFunctionAndLogException2();
-			values2.put(key, reduced2);
-			collector.collect(coReducer.map2(reduced2));
-		} else {
-			values2.put(key, nextValue2);
-			collector.collect(coReducer.map2(nextValue2));
+	protected void handleStream2() throws Exception {
+		while (windowStart2()) {
+			circularList2.newSlide();
 		}
+		while (windowEnd2()) {
+			reduce2();
+			circularList2.shiftWindow();
+		}
+		circularList2.add(reuse2);
+	}
+
+	protected void reduce1() throws Exception {
+		userIterator1.load(circularList1.getIterator());
+		callUserFunctionAndLogException1();
+	}
+
+	protected void reduce2() throws Exception {
+		userIterator2.load(circularList2.getIterator());
+		callUserFunctionAndLogException2();
 	}
 
 	@Override
 	protected void callUserFunction1() throws Exception {
-		reduced1 = coReducer.reduce1(currentValue1, nextValue1);
-
+		coReducer.reduce1(userIterable1, collector);
 	}
 
 	@Override
 	protected void callUserFunction2() throws Exception {
-		reduced2 = coReducer.reduce2(currentValue2, nextValue2);
+		coReducer.reduce2(userIterable2, collector);
+	}
+
+	protected abstract boolean windowStart1() throws Exception;
+
+	protected abstract boolean windowStart2() throws Exception;
+
+	protected abstract boolean windowEnd1() throws Exception;
+
+	protected abstract boolean windowEnd2() throws Exception;
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		userIterable1 = new BatchIterable1();
+		userIterable2 = new BatchIterable2();
+	}
+
+	protected class BatchIterable1 implements Iterable<IN1> {
+		@Override
+		public Iterator<IN1> iterator() {
+			return userIterator1;
+		}
+	}
 
+	protected class BatchIterable2 implements Iterable<IN2> {
+		@Override
+		public Iterator<IN2> iterator() {
+			return userIterator2;
+		}
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedBatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedBatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedBatchGroupReduceInvokable.java
new file mode 100644
index 0000000..20679dd
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedBatchGroupReduceInvokable.java
@@ -0,0 +1,99 @@
+/**
+ * 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.co;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class CoGroupedBatchGroupReduceInvokable<IN1, IN2, OUT> extends
+		CoBatchGroupReduceInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	private int keyPosition1;
+	private int keyPosition2;
+	private Iterator<StreamRecord<IN1>> iterator1;
+	private Iterator<StreamRecord<IN2>> iterator2;
+	private MutableTableState<Object, List<IN1>> values1;
+	private MutableTableState<Object, List<IN2>> values2;
+	private IN1 nextValue1;
+	private IN2 nextValue2;
+
+	public CoGroupedBatchGroupReduceInvokable(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReduceFunction, long batchSize1,
+			long batchSize2, long slideSize1, long slideSize2, int keyPosition1, int keyPosition2) {
+		super(coReduceFunction, batchSize1, batchSize2, slideSize1, slideSize2);
+		this.keyPosition1 = keyPosition1;
+		this.keyPosition2 = keyPosition2;
+		values1 = new MutableTableState<Object, List<IN1>>();
+		values2 = new MutableTableState<Object, List<IN2>>();
+	}
+
+	@Override
+	protected void reduce1() {
+		iterator1 = circularList1.getIterator();
+		while (iterator1.hasNext()) {
+			StreamRecord<IN1> nextRecord = iterator1.next();
+			Object key = nextRecord.getField(keyPosition1);
+			nextValue1 = nextRecord.getObject();
+
+			List<IN1> group = values1.get(key);
+			if (group != null) {
+				group.add(nextValue1);
+			} else {
+				group = new ArrayList<IN1>();
+				group.add(nextValue1);
+				values1.put(key, group);
+			}
+		}
+		for (List<IN1> group : values1.values()) {
+			userIterable1 = group;
+			callUserFunctionAndLogException1();
+		}
+		values1.clear();
+	}
+
+	@Override
+	protected void reduce2() {
+		iterator2 = circularList2.getIterator();
+		while (iterator2.hasNext()) {
+			StreamRecord<IN2> nextRecord = iterator2.next();
+			Object key = nextRecord.getField(keyPosition2);
+			nextValue2 = nextRecord.getObject();
+
+			List<IN2> group = values2.get(key);
+			if (group != null) {
+				group.add(nextValue2);
+			} else {
+				group = new ArrayList<IN2>();
+				group.add(nextValue2);
+				values2.put(key, group);
+			}
+		}
+		for (List<IN2> group : values2.values()) {
+			userIterable2 = group;
+			callUserFunctionAndLogException2();
+		}
+		values2.clear();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedReduceInvokable.java
new file mode 100644
index 0000000..0daa1ea
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedReduceInvokable.java
@@ -0,0 +1,85 @@
+/**
+ * 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.co;
+
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class CoGroupedReduceInvokable<IN1, IN2, OUT> extends CoStreamReduceInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	private int keyPosition1;
+	private int keyPosition2;
+	private MutableTableState<Object, IN1> values1;
+	private MutableTableState<Object, IN2> values2;
+	IN1 reduced1;
+	IN2 reduced2;
+
+	public CoGroupedReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer, int keyPosition1,
+			int keyPosition2) {
+		super(coReducer);
+		this.coReducer = coReducer;
+		this.keyPosition1 = keyPosition1;
+		this.keyPosition2 = keyPosition2;
+		values1 = new MutableTableState<Object, IN1>();
+		values2 = new MutableTableState<Object, IN2>();
+	}
+
+	@Override
+	public void handleStream1() throws Exception {
+		Object key = reuse1.getField(keyPosition1);
+		currentValue1 = values1.get(key);
+		nextValue1 = reuse1.getObject();
+		if (currentValue1 != null) {
+			callUserFunctionAndLogException1();
+			values1.put(key, reduced1);
+			collector.collect(coReducer.map1(reduced1));
+		} else {
+			values1.put(key, nextValue1);
+			collector.collect(coReducer.map1(nextValue1));
+		}
+	}
+
+	@Override
+	public void handleStream2() throws Exception {
+		Object key = reuse2.getField(keyPosition2);
+		currentValue2 = values2.get(key);
+		nextValue2 = reuse2.getObject();
+		if (currentValue2 != null) {
+			callUserFunctionAndLogException2();
+			values2.put(key, reduced2);
+			collector.collect(coReducer.map2(reduced2));
+		} else {
+			values2.put(key, nextValue2);
+			collector.collect(coReducer.map2(nextValue2));
+		}
+	}
+
+	@Override
+	protected void callUserFunction1() throws Exception {
+		reduced1 = coReducer.reduce1(currentValue1, nextValue1);
+
+	}
+
+	@Override
+	protected void callUserFunction2() throws Exception {
+		reduced2 = coReducer.reduce2(currentValue2, nextValue2);
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedWindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedWindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedWindowGroupReduceInvokable.java
new file mode 100644
index 0000000..60cace6
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupedWindowGroupReduceInvokable.java
@@ -0,0 +1,102 @@
+/**
+ * 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.co;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class CoGroupedWindowGroupReduceInvokable<IN1, IN2, OUT> extends
+		CoWindowGroupReduceInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	private int keyPosition1;
+	private int keyPosition2;
+	private Iterator<StreamRecord<IN1>> iterator1;
+	private Iterator<StreamRecord<IN2>> iterator2;
+	private MutableTableState<Object, List<IN1>> values1;
+	private MutableTableState<Object, List<IN2>> values2;
+	private IN1 nextValue1;
+	private IN2 nextValue2;
+
+	public CoGroupedWindowGroupReduceInvokable(
+			CoGroupReduceFunction<IN1, IN2, OUT> coReduceFunction, long windowSize1,
+			long windowSize2, long slideInterval1, long slideInterval2, int keyPosition1,
+			int keyPosition2, TimeStamp<IN1> timestamp1, TimeStamp<IN2> timestamp2) {
+		super(coReduceFunction, windowSize1, windowSize2, slideInterval1, slideInterval2,
+				timestamp1, timestamp2);
+		this.keyPosition1 = keyPosition1;
+		this.keyPosition2 = keyPosition2;
+		values1 = new MutableTableState<Object, List<IN1>>();
+		values2 = new MutableTableState<Object, List<IN2>>();
+	}
+
+	@Override
+	protected void reduce1() {
+		iterator1 = circularList1.getIterator();
+		while (iterator1.hasNext()) {
+			StreamRecord<IN1> nextRecord = iterator1.next();
+			Object key = nextRecord.getField(keyPosition1);
+			nextValue1 = nextRecord.getObject();
+
+			List<IN1> group = values1.get(key);
+			if (group != null) {
+				group.add(nextValue1);
+			} else {
+				group = new ArrayList<IN1>();
+				group.add(nextValue1);
+				values1.put(key, group);
+			}
+		}
+		for (List<IN1> group : values1.values()) {
+			userIterable1 = group;
+			callUserFunctionAndLogException1();
+		}
+		values1.clear();
+	}
+
+	@Override
+	protected void reduce2() {
+		iterator2 = circularList2.getIterator();
+		while (iterator2.hasNext()) {
+			StreamRecord<IN2> nextRecord = iterator2.next();
+			Object key = nextRecord.getField(keyPosition2);
+			nextValue2 = nextRecord.getObject();
+
+			List<IN2> group = values2.get(key);
+			if (group != null) {
+				group.add(nextValue2);
+			} else {
+				group = new ArrayList<IN2>();
+				group.add(nextValue2);
+				values2.put(key, group);
+			}
+		}
+		for (List<IN2> group : values2.values()) {
+			userIterable2 = group;
+			callUserFunctionAndLogException2();
+		}
+		values2.clear();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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 b064df7..0e1f2b0 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
@@ -85,9 +85,11 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 			if (next == 0) {
 				break;
 			} else if (next == 1) {
+				initialize1();
 				handleStream1();
 				resetReuse1();
 			} else {
+				initialize2();
 				handleStream2();
 				resetReuse2();
 			}
@@ -100,8 +102,10 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 			if (next == 0) {
 				break;
 			} else if (next == 1) {
+				initialize1();
 				handleStream1();
 			} else {
+				initialize2();
 				handleStream2();
 			}
 		}
@@ -114,7 +118,15 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 	protected abstract void callUserFunction1() throws Exception;
 
 	protected abstract void callUserFunction2() throws Exception;
-	
+
+	protected void initialize1() {
+
+	};
+
+	protected void initialize2() {
+
+	};
+
 	protected void callUserFunctionAndLogException1() {
 		try {
 			callUserFunction1();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
deleted file mode 100644
index 407f217..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
+++ /dev/null
@@ -1,70 +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.operator.co;
-
-import org.apache.flink.streaming.api.function.co.CoReduceFunction;
-
-public class CoReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
-	private static final long serialVersionUID = 1L;
-
-	protected CoReduceFunction<IN1, IN2, OUT> coReducer;
-	protected IN1 currentValue1 = null;
-	protected IN2 currentValue2 = null;
-	protected IN1 nextValue1 = null;
-	protected IN2 nextValue2 = null;
-
-	public CoReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer) {
-		super(coReducer);
-		this.coReducer = coReducer;
-		currentValue1 = null;
-		currentValue2 = null;
-	}
-
-	@Override
-	public void handleStream1() throws Exception {
-		nextValue1 = reuse1.getObject();
-		callUserFunctionAndLogException1();
-	}
-
-	@Override
-	public void handleStream2() throws Exception {
-		nextValue2 = reuse2.getObject();
-		callUserFunctionAndLogException2();
-	}
-
-	@Override
-	protected void callUserFunction1() throws Exception {
-		if (currentValue1 != null) {
-			currentValue1 = coReducer.reduce1(currentValue1, nextValue1);
-		} else {
-			currentValue1 = nextValue1;
-		}
-		collector.collect(coReducer.map1(currentValue1));
-	}
-
-	@Override
-	protected void callUserFunction2() throws Exception {
-		if (currentValue2 != null) {
-			currentValue2 = coReducer.reduce2(currentValue2, nextValue2);
-		} else {
-			currentValue2 = nextValue2;
-		}
-		collector.collect(coReducer.map2(currentValue2));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
new file mode 100644
index 0000000..81a6a23
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator.co;
+
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+
+public class CoStreamReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	protected CoReduceFunction<IN1, IN2, OUT> coReducer;
+	protected IN1 currentValue1 = null;
+	protected IN2 currentValue2 = null;
+	protected IN1 nextValue1 = null;
+	protected IN2 nextValue2 = null;
+
+	public CoStreamReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer) {
+		super(coReducer);
+		this.coReducer = coReducer;
+		currentValue1 = null;
+		currentValue2 = null;
+	}
+
+	@Override
+	public void handleStream1() throws Exception {
+		nextValue1 = reuse1.getObject();
+		callUserFunctionAndLogException1();
+	}
+
+	@Override
+	public void handleStream2() throws Exception {
+		nextValue2 = reuse2.getObject();
+		callUserFunctionAndLogException2();
+	}
+
+	@Override
+	protected void callUserFunction1() throws Exception {
+		if (currentValue1 != null) {
+			currentValue1 = coReducer.reduce1(currentValue1, nextValue1);
+		} else {
+			currentValue1 = nextValue1;
+		}
+		collector.collect(coReducer.map1(currentValue1));
+	}
+
+	@Override
+	protected void callUserFunction2() throws Exception {
+		if (currentValue2 != null) {
+			currentValue2 = coReducer.reduce2(currentValue2, nextValue2);
+		} else {
+			currentValue2 = nextValue2;
+		}
+		collector.collect(coReducer.map2(currentValue2));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoWindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoWindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoWindowGroupReduceInvokable.java
new file mode 100644
index 0000000..49c73b5
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoWindowGroupReduceInvokable.java
@@ -0,0 +1,98 @@
+/**
+ * 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.co;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
+
+public class CoWindowGroupReduceInvokable<IN1, IN2, OUT> extends CoGroupReduceInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	protected long startTime1;
+	protected long startTime2;
+	protected long endTime1;
+	protected long endTime2;
+	protected long currentTime;
+	protected TimeStamp<IN1> timestamp1;
+	protected TimeStamp<IN2> timestamp2;
+
+	public CoWindowGroupReduceInvokable(CoGroupReduceFunction<IN1, IN2, OUT> reduceFunction,
+			long windowSize1, long windowSize2, long slideInterval1, long slideInterval2,
+			TimeStamp<IN1> timestamp1, TimeStamp<IN2> timestamp2) {
+		super(reduceFunction, windowSize1, windowSize2, slideInterval1, slideInterval2);
+		this.timestamp1 = timestamp1;
+		this.timestamp2 = timestamp2;
+		startTime1 = timestamp1.getStartTime();
+		startTime2 = timestamp2.getStartTime();
+		endTime1 = startTime1 + windowSize1;
+		endTime2 = startTime2 + windowSize2;
+	}
+
+	@Override
+	protected boolean windowStart1() throws Exception {
+		if (currentTime - startTime1 >= slideInterval1) {
+			startTime1 += slideInterval1;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowStart2() throws Exception {
+		if (currentTime - startTime2 >= slideInterval2) {
+			startTime2 += slideInterval2;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowEnd1() throws Exception {
+		if (currentTime >= endTime1) {
+			endTime1 += slideInterval1;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected boolean windowEnd2() throws Exception {
+		if (currentTime >= endTime2) {
+			endTime2 += slideInterval2;
+			return true;
+		}
+		return false;
+	}
+
+	@Override
+	protected void initialize1() {
+		currentTime = timestamp1.getTimestamp(reuse1.getObject());
+	}
+
+	@Override
+	protected void initialize2() {
+		currentTime = timestamp2.getTimestamp(reuse2.getObject());
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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
new file mode 100644
index 0000000..46ccce7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
@@ -0,0 +1,67 @@
+/**
+ * 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.state;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+
+/**
+ * A simple class, that manages a circular queue with sliding interval. If the
+ * queue if full and a new element is added, the elements that belong to the
+ * first sliding interval are removed.
+ */
+public class CircularFifoList<T> {
+	private Queue<T> queue;
+	private Deque<Long> slideSizes;
+	private long counter;
+
+	public CircularFifoList() {
+		this.queue = new LinkedList<T>();
+		this.slideSizes = new ArrayDeque<Long>();
+		this.counter = 0;
+	}
+
+	public void add(T element) {
+		queue.add(element);
+		counter++;
+	}
+
+	public void newSlide() {
+		slideSizes.add(counter);
+		counter = 0;
+	}
+
+	public void shiftWindow() {
+		for (int i = 0; i < slideSizes.getFirst(); i++) {
+			queue.remove();
+		}
+		slideSizes.remove();
+	}
+
+	public Iterator<T> getIterator() {
+		return queue.iterator();
+	}
+
+	@Override
+	public String toString() {
+		return queue.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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
new file mode 100644
index 0000000..e3ba4b5
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/StreamIterator.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.state;
+
+import java.util.Iterator;
+
+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> {
+	private Iterator<StreamRecord<T>> iterator = null;
+
+	public void load(Iterator<StreamRecord<T>> iterator) {
+		this.iterator = iterator;
+	}
+
+	@Override
+	public boolean hasNext() {
+		return iterator.hasNext();
+	}
+
+	@Override
+	public T next() {
+		return iterator.next().getObject();
+	}
+
+	@Override
+	public void remove() {
+		iterator.remove();
+	}
+
+	@Override
+	public String toString() {
+		return iterator.toString();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/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 3861aab..54264fe 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
@@ -28,7 +28,7 @@ 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.MinAggregationFunction;
 import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
-import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
 import org.apache.flink.streaming.util.MockInvokable;
 import org.junit.Test;
@@ -107,13 +107,13 @@ public class AggregationFunctionTest {
 				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxFunction), getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedSumList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedMinList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedMaxList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), getInputList());
+				new GroupedReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), getInputList());
 
 		assertEquals(expectedSumList, sumList);
 		assertEquals(expectedMinList, minList);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoBatchGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoBatchGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoBatchGroupReduceTest.java
new file mode 100644
index 0000000..470d956
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoBatchGroupReduceTest.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.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoBatchGroupReduceInvokable;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class CoBatchGroupReduceTest {
+
+	public static final class MyCoGroupReduceFunction implements
+			CoGroupReduceFunction<Integer, String, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce1(Iterable<Integer> values, Collector<String> out) throws Exception {
+			String gather = "";
+			for (Integer value : values) {
+				gather += value.toString();
+			}
+			out.collect(gather);
+		}
+
+		@Override
+		public void reduce2(Iterable<String> values, Collector<String> out) throws Exception {
+			String gather = "";
+			for (String value : values) {
+				gather += value;
+			}
+			out.collect(gather);
+		}
+	}
+
+	@Test
+	public void coBatchGroupReduceTest1() {
+
+		List<Integer> inputs1 = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+
+		List<String> inputs2 = Arrays.asList("a", "b", "c", "d", "e", "f", "g", "h");
+
+		CoBatchGroupReduceInvokable<Integer, String, String> invokable = new CoBatchGroupReduceInvokable<Integer, String, String>(
+				new MyCoGroupReduceFunction(), 4L, 2L, 4L, 2L);
+
+		List<String> expected = Arrays.asList("1234", "5678", "ab", "cd", "ef", "gh");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+	@Test
+	public void coBatchGroupReduceTest2() {
+
+		List<Integer> inputs1 = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+
+		List<String> inputs2 = Arrays.asList("a", "b", "c", "d", "e", "f", "g", "h");
+
+		CoBatchGroupReduceInvokable<Integer, String, String> invokable = new CoBatchGroupReduceInvokable<Integer, String, String>(
+				new MyCoGroupReduceFunction(), 4L, 2L, 3L, 1L);
+
+		List<String> expected = Arrays.asList("1234", "4567", "78910", "ab", "bc", "cd", "de",
+				"ef", "fg", "gh");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
index acf2f28..264b074 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
@@ -1,4 +1,5 @@
-/** Licensed to the Apache Software Foundation (ASF) under one or more
+/**
+ * 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
@@ -12,7 +13,6 @@
  * 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;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupReduceTest.java
deleted file mode 100644
index bf9a772..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupReduceTest.java
+++ /dev/null
@@ -1,96 +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.operator;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.function.co.CoReduceFunction;
-import org.apache.flink.streaming.api.invokable.operator.co.CoGroupReduceInvokable;
-import org.apache.flink.streaming.util.MockCoInvokable;
-import org.junit.Test;
-
-public class CoGroupReduceTest {
-	
-	private final static class MyCoReduceFunction implements
-			CoReduceFunction<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple3<String, String, String> reduce1(Tuple3<String, String, String> value1,
-				Tuple3<String, String, String> value2) {
-			return new Tuple3<String, String, String>(value1.f0, value1.f1 + value2.f1, value1.f2);
-		}
-
-		@Override
-		public Tuple2<Integer, Integer> reduce2(Tuple2<Integer, Integer> value1,
-				Tuple2<Integer, Integer> value2) {
-			return new Tuple2<Integer, Integer>(value1.f0, value1.f1 + value2.f1);
-		}
-
-		@Override
-		public String map1(Tuple3<String, String, String> value) {
-			return value.f1;
-		}
-
-		@Override
-		public String map2(Tuple2<Integer, Integer> value) {
-			return value.f1.toString();
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void coGroupReduceTest() {
-		Tuple3<String, String, String> word1 = new Tuple3<String, String, String>("a", "word1", "b");
-		Tuple3<String, String, String> word2 = new Tuple3<String, String, String>("b", "word2", "a");
-		Tuple3<String, String, String> word3 = new Tuple3<String, String, String>("a", "word3", "a");
-		Tuple2<Integer, Integer> int1 = new Tuple2<Integer, Integer>(2, 1);
-		Tuple2<Integer, Integer> int2 = new Tuple2<Integer, Integer>(1, 2);
-		Tuple2<Integer, Integer> int3 = new Tuple2<Integer, Integer>(0, 3);
-		Tuple2<Integer, Integer> int4 = new Tuple2<Integer, Integer>(2, 4);
-		Tuple2<Integer, Integer> int5 = new Tuple2<Integer, Integer>(1, 5);
-		
-		CoGroupReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> invokable = new CoGroupReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
-				new MyCoReduceFunction(), 0, 0);
-
-		List<String> expected = Arrays.asList("word1", "1", "word2", "2", "word1word3", "3", "5",
-				"7");
-
-		List<String> actualList = MockCoInvokable.createAndExecute(invokable,
-				Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5));
-
-		assertEquals(expected, actualList);
-		
-	
-		invokable = new CoGroupReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
-				new MyCoReduceFunction(), 2, 0);
-
-		expected = Arrays.asList("word1", "1", "word2", "2", "word2word3", "3", "5",
-				"7");
-		
-		actualList = MockCoInvokable.createAndExecute(invokable,
-				Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5));
-		
-		assertEquals(expected, actualList);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedBatchGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedBatchGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedBatchGroupReduceTest.java
new file mode 100644
index 0000000..f150d30
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedBatchGroupReduceTest.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedBatchGroupReduceInvokable;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class CoGroupedBatchGroupReduceTest {
+
+	public static final class MyCoGroupReduceFunction implements
+			CoGroupReduceFunction<Tuple2<Integer, Integer>, Tuple2<Integer, String>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce1(Iterable<Tuple2<Integer, Integer>> values, Collector<String> out)
+				throws Exception {
+			String gather = "";
+			for (Tuple2<Integer, Integer> value : values) {
+				gather += value.f1.toString();
+			}
+			out.collect(gather);
+		}
+
+		@Override
+		public void reduce2(Iterable<Tuple2<Integer, String>> values, Collector<String> out)
+				throws Exception {
+			String gather = "";
+			for (Tuple2<Integer, String> value : values) {
+				gather += value.f1;
+			}
+			out.collect(gather);
+		}
+	}
+
+	final static int KEY_POSITION1 = 0;
+	final static int KEY_POSITION2 = 0;
+
+	@Test
+	public void coGroupedBatchGroupReduceTest1() {
+
+		List<Tuple2<Integer, Integer>> inputs1 = new ArrayList<Tuple2<Integer, Integer>>();
+		for (Integer i = 1; i <= 10; i++) {
+			inputs1.add(new Tuple2<Integer, Integer>(i % 3, i));
+		}
+
+		List<Tuple2<Integer, String>> inputs2 = new ArrayList<Tuple2<Integer, String>>();
+		for (char ch = 'a'; ch <= 'h'; ch++) {
+			inputs2.add(new Tuple2<Integer, String>(((int) ch) % 3, ch + ""));
+		}
+
+		CoGroupedBatchGroupReduceInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, String>, String> invokable = new CoGroupedBatchGroupReduceInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, String>, String>(
+				new MyCoGroupReduceFunction(), 5L, 4L, 5L, 4L, KEY_POSITION1, KEY_POSITION2);
+
+		List<String> expected = Arrays.asList("14", "25", "3", "69", "710", "8", "ad", "b", "c",
+				"eh", "f", "g");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+	@Test
+	public void coGroupedBatchGroupReduceTest2() {
+
+		List<Tuple2<Integer, Integer>> inputs1 = new ArrayList<Tuple2<Integer, Integer>>();
+		for (Integer i = 1; i <= 10; i++) {
+			inputs1.add(new Tuple2<Integer, Integer>(i % 3, i));
+		}
+
+		List<Tuple2<Integer, String>> inputs2 = new ArrayList<Tuple2<Integer, String>>();
+		for (char ch = 'a'; ch <= 'h'; ch++) {
+			inputs2.add(new Tuple2<Integer, String>(((int) ch) % 3, ch + ""));
+		}
+
+		CoGroupedBatchGroupReduceInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, String>, String> invokable = new CoGroupedBatchGroupReduceInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, String>, String>(
+				new MyCoGroupReduceFunction(), 6L, 6L, 3L, 2L, KEY_POSITION1, KEY_POSITION2);
+
+		List<String> expected = Arrays.asList("14", "25", "36", "47", "58", "69", "ad", "be", "cf",
+				"cf", "dg", "eh");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedReduceTest.java
new file mode 100644
index 0000000..9842d1c
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedReduceTest.java
@@ -0,0 +1,94 @@
+/**
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedReduceInvokable;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.junit.Test;
+
+public class CoGroupedReduceTest {
+
+	private final static class MyCoReduceFunction implements
+			CoReduceFunction<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple3<String, String, String> reduce1(Tuple3<String, String, String> value1,
+				Tuple3<String, String, String> value2) {
+			return new Tuple3<String, String, String>(value1.f0, value1.f1 + value2.f1, value1.f2);
+		}
+
+		@Override
+		public Tuple2<Integer, Integer> reduce2(Tuple2<Integer, Integer> value1,
+				Tuple2<Integer, Integer> value2) {
+			return new Tuple2<Integer, Integer>(value1.f0, value1.f1 + value2.f1);
+		}
+
+		@Override
+		public String map1(Tuple3<String, String, String> value) {
+			return value.f1;
+		}
+
+		@Override
+		public String map2(Tuple2<Integer, Integer> value) {
+			return value.f1.toString();
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void coGroupedReduceTest() {
+		Tuple3<String, String, String> word1 = new Tuple3<String, String, String>("a", "word1", "b");
+		Tuple3<String, String, String> word2 = new Tuple3<String, String, String>("b", "word2", "a");
+		Tuple3<String, String, String> word3 = new Tuple3<String, String, String>("a", "word3", "a");
+		Tuple2<Integer, Integer> int1 = new Tuple2<Integer, Integer>(2, 1);
+		Tuple2<Integer, Integer> int2 = new Tuple2<Integer, Integer>(1, 2);
+		Tuple2<Integer, Integer> int3 = new Tuple2<Integer, Integer>(0, 3);
+		Tuple2<Integer, Integer> int4 = new Tuple2<Integer, Integer>(2, 4);
+		Tuple2<Integer, Integer> int5 = new Tuple2<Integer, Integer>(1, 5);
+
+		CoGroupedReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> invokable = new CoGroupedReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
+				new MyCoReduceFunction(), 0, 0);
+
+		List<String> expected = Arrays.asList("word1", "1", "word2", "2", "word1word3", "3", "5",
+				"7");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable,
+				Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5));
+
+		assertEquals(expected, actualList);
+
+		invokable = new CoGroupedReduceInvokable<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
+				new MyCoReduceFunction(), 2, 0);
+
+		expected = Arrays.asList("word1", "1", "word2", "2", "word2word3", "3", "5", "7");
+
+		actualList = MockCoInvokable.createAndExecute(invokable,
+				Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5));
+
+		assertEquals(expected, actualList);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedWindowGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedWindowGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedWindowGroupReduceTest.java
new file mode 100644
index 0000000..b421284
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoGroupedWindowGroupReduceTest.java
@@ -0,0 +1,223 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedWindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class CoGroupedWindowGroupReduceTest {
+
+	public static final class MyCoGroupReduceFunction1 implements
+			CoGroupReduceFunction<Character, Character, String> {
+		private static final long serialVersionUID = 1L;
+
+		@SuppressWarnings("unused")
+		@Override
+		public void reduce1(Iterable<Character> values, Collector<String> out) throws Exception {
+			Integer gather = 0;
+			Character ch = values.iterator().next();
+			for (Character value : values) {
+				gather++;
+			}
+			out.collect(ch + ":" + gather);
+		}
+
+		@SuppressWarnings("unused")
+		@Override
+		public void reduce2(Iterable<Character> values, Collector<String> out) throws Exception {
+			Integer gather = 0;
+			Character ch = values.iterator().next();
+			for (Character value : values) {
+				gather++;
+			}
+			out.collect(ch + ":" + gather);
+		}
+	}
+
+	public static final class MyCoGroupReduceFunction2 implements
+			CoGroupReduceFunction<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce1(Iterable<Tuple2<String, Integer>> values, Collector<String> out)
+				throws Exception {
+			String gather = "";
+			for (Tuple2<String, Integer> value : values) {
+				gather += value.f0;
+			}
+			out.collect(gather);
+		}
+
+		@Override
+		public void reduce2(Iterable<Tuple2<Integer, Integer>> values, Collector<String> out)
+				throws Exception {
+			Integer gather = 0;
+			for (Tuple2<Integer, Integer> value : values) {
+				gather += value.f0;
+			}
+			out.collect(gather.toString());
+		}
+	}
+
+	public static final class MyTimeStamp<T> implements TimeStamp<T> {
+		private static final long serialVersionUID = 1L;
+
+		private Iterator<Long> timestamps;
+		private long start;
+
+		public MyTimeStamp(List<Long> timestamps) {
+			this.timestamps = timestamps.iterator();
+			this.start = timestamps.get(0);
+		}
+
+		@Override
+		public long getTimestamp(T value) {
+			long ts = timestamps.next();
+			return ts;
+		}
+
+		@Override
+		public long getStartTime() {
+			return start;
+		}
+	}
+
+	@Test
+	public void coGroupedWindowGroupReduceTest1() {
+
+		List<Character> inputs1 = new ArrayList<Character>();
+		inputs1.add('a');
+		inputs1.add('b');
+		inputs1.add('c');
+		inputs1.add('a');
+		inputs1.add('a');
+		inputs1.add('c');
+		inputs1.add('b');
+		inputs1.add('c');
+		inputs1.add('a');
+		inputs1.add('a');
+		inputs1.add('x');
+
+		List<Character> inputs2 = new ArrayList<Character>();
+		inputs2.add('a');
+		inputs2.add('d');
+		inputs2.add('d');
+		inputs2.add('e');
+		inputs2.add('d');
+		inputs2.add('e');
+		inputs2.add('e');
+		inputs2.add('a');
+		inputs2.add('a');
+		inputs2.add('x');
+
+		List<Long> timestamps1 = Arrays.asList(0L, 1L, 1L, 1L, 2L, 2L, 2L, 3L, 8L, 10L, 11L);
+
+		List<Long> timestamps2 = Arrays.asList(0L, 5L, 5L, 6L, 6L, 7L, 7L, 8L, 8L, 10L);
+
+		CoGroupedWindowGroupReduceInvokable<Character, Character, String> invokable = new CoGroupedWindowGroupReduceInvokable<Character, Character, String>(
+				new MyCoGroupReduceFunction1(), 5L, 5L, 3L, 5L, 0, 0, new MyTimeStamp<Character>(
+						timestamps1), new MyTimeStamp<Character>(timestamps2));
+
+		List<String> expected = new ArrayList<String>();
+		expected.add("a:3");
+		expected.add("b:2");
+		expected.add("c:3");
+		expected.add("c:1");
+		expected.add("a:2");
+		expected.add("a:1");
+		expected.add("a:2");
+		expected.add("d:3");
+		expected.add("e:3");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+	@Test
+	public void coGroupedWindowGroupReduceTest2() {
+
+		List<Tuple2<String, Integer>> inputs1 = new ArrayList<Tuple2<String, Integer>>();
+		inputs1.add(new Tuple2<String, Integer>("a", 1));
+		inputs1.add(new Tuple2<String, Integer>("b", 1));
+		inputs1.add(new Tuple2<String, Integer>("c", 0));
+		inputs1.add(new Tuple2<String, Integer>("d", 0));
+		inputs1.add(new Tuple2<String, Integer>("e", 1));
+		inputs1.add(new Tuple2<String, Integer>("f", 1));
+		inputs1.add(new Tuple2<String, Integer>("g", 0));
+		inputs1.add(new Tuple2<String, Integer>("h", 0));
+		inputs1.add(new Tuple2<String, Integer>("i", 1));
+		inputs1.add(new Tuple2<String, Integer>("j", 1));
+
+		List<Tuple2<Integer, Integer>> inputs2 = new ArrayList<Tuple2<Integer, Integer>>();
+		inputs2.add(new Tuple2<Integer, Integer>(1, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(2, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(3, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(4, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(5, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(6, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(7, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(8, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(9, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(10, 2));
+
+		List<Long> timestamps1 = Arrays.asList(0L, 1L, 1L, 1L, 2L, 2L, 2L, 3L, 4L, 7L);
+
+		List<Long> timestamps2 = Arrays.asList(0L, 5L, 5L, 6L, 6L, 7L, 7L, 8L, 8L, 10L);
+
+		CoGroupedWindowGroupReduceInvokable<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String> invokable = new CoGroupedWindowGroupReduceInvokable<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String>(
+				new MyCoGroupReduceFunction2(), 2L, 4L, 2L, 2L, 1, 1,
+				new MyTimeStamp<Tuple2<String, Integer>>(timestamps1),
+				new MyTimeStamp<Tuple2<Integer, Integer>>(timestamps2));
+
+		List<String> expected = new ArrayList<String>();
+		expected.add("ab");
+		expected.add("cd");
+		expected.add("ef");
+		expected.add("gh");
+		expected.add("i");
+		expected.add("1");
+		expected.add("3");
+		expected.add("2");
+		expected.add("15");
+		expected.add("12");
+		expected.add("21");
+		expected.add("18");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java
deleted file mode 100755
index 4f52444..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.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.invokable.operator;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.streaming.api.function.co.CoReduceFunction;
-import org.apache.flink.streaming.api.invokable.operator.co.CoReduceInvokable;
-import org.apache.flink.streaming.util.MockCoInvokable;
-import org.junit.Test;
-
-public class CoReduceTest {
-
-	public static class MyCoReduceFunction implements CoReduceFunction<Integer, String, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce1(Integer value1, Integer value2) {
-			return value1 * value2;
-		}
-
-		@Override
-		public String reduce2(String value1, String value2) {
-			return value1 + value2;
-		}
-
-		@Override
-		public Integer map1(Integer value) {
-			return value;
-		}
-
-		@Override
-		public Integer map2(String value) {
-			return Integer.parseInt(value);
-		}
-
-	}
-
-	@Test
-	public void coGroupReduceTest() {
-
-		CoReduceInvokable<Integer, String, Integer> coReduce = new CoReduceInvokable<Integer, String, Integer>(
-				new MyCoReduceFunction());
-
-		List<Integer> expected1 = Arrays.asList(1, 9, 2, 99, 6, 998, 24);
-
-		assertEquals(
-				expected1,
-				(MockCoInvokable.createAndExecute(coReduce, Arrays.asList(1, 2, 3, 4),
-						Arrays.asList("9", "9", "8"))));
-
-	}
-}


[11/18] git commit: [streaming] Added checked exception to execution

Posted by mb...@apache.org.
[streaming] Added checked exception to execution


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

Branch: refs/heads/master
Commit: 48d8ed703e9d8a25f2143514e8f9cad66dc2593d
Parents: 03a28cb
Author: ghermann <re...@gmail.com>
Authored: Tue Sep 16 13:06:49 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 .../connectors/kafka/KafkaTopology.java         |  2 +-
 .../connectors/twitter/TwitterLocal.java        |  2 +-
 .../connectors/twitter/TwitterStreaming.java    |  2 +-
 .../api/environment/LocalStreamEnvironment.java |  4 +--
 .../environment/StreamExecutionEnvironment.java |  3 ++-
 .../flink/streaming/util/ClusterUtil.java       | 11 ++++-----
 .../streamcomponent/StreamComponentTest.java    | 26 +++++++++-----------
 .../examples/basictopology/BasicTopology.java   |  2 +-
 .../examples/cellinfo/CellInfoLocal.java        |  2 +-
 .../streaming/examples/join/JoinLocal.java      |  2 +-
 .../ml/IncrementalLearningSkeleton.java         |  2 +-
 .../examples/window/join/WindowJoinLocal.java   |  2 +-
 .../examples/wordcount/WordCountLocal.java      |  2 +-
 13 files changed, 29 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/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 64ea810..998a395 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
@@ -90,7 +90,7 @@ public class KafkaTopology {
 
 	private static final int SOURCE_PARALELISM = 1;
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 465a500..6f739b3 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -58,7 +58,7 @@ public class TwitterLocal {
 
 	}
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		String path = new String();
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
index 5927ce4..540895c 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -72,7 +72,7 @@ public class TwitterStreaming {
 		}
 	}
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		String path = new String();
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/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 828a566..f39a20a 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
@@ -26,11 +26,11 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	 * 
 	 */
 	@Override
-	public void execute() {
+	public void execute() throws Exception {
 		ClusterUtil.runOnMiniCluster(this.jobGraphBuilder.getJobGraph(), getExecutionParallelism());
 	}
 
-	public void executeTest(long memorySize) {
+	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/48d8ed70/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 6187c99..ad379b3 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
@@ -454,8 +454,9 @@ public abstract class StreamExecutionEnvironment {
 	 * <p>
 	 * The program execution will be logged and displayed with a generated
 	 * default name.
+	 * @throws Exception 
 	 **/
-	public abstract void execute();
+	public abstract void execute() throws Exception;
 
 	/**
 	 * Getter of the {@link JobGraphBuilder} of the streaming job.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
index 0853ec7..ec73aef 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
@@ -42,7 +42,7 @@ public class ClusterUtil {
 	 * @param memorySize
 	 *            memorySize
 	 */
-	public static void runOnMiniCluster(JobGraph jobGraph, int numberOfTaskTrackers, long memorySize) {
+	public static void runOnMiniCluster(JobGraph jobGraph, int numberOfTaskTrackers, long memorySize) throws Exception  {
 
 		Configuration configuration = jobGraph.getJobConfiguration();
 
@@ -59,15 +59,14 @@ public class ClusterUtil {
 			Client client = new Client(new InetSocketAddress("localhost",
 					exec.getJobManagerRpcPort()), configuration, ClusterUtil.class.getClassLoader());
 			client.run(jobGraph, true);
-
 		} catch (ProgramInvocationException e) {
 			if (e.getMessage().contains("GraphConversionException")) {
-				throw new RuntimeException(CANNOT_EXECUTE_EMPTY_JOB, e);
+				throw new Exception(CANNOT_EXECUTE_EMPTY_JOB, e);
 			} else {
-				throw new RuntimeException(e.getMessage(), e);
+				throw e;
 			}
 		} catch (Exception e) {
-			throw new RuntimeException(e.getMessage(), e);
+			throw e;
 		} finally {
 			try {
 				exec.stop();
@@ -76,7 +75,7 @@ public class ClusterUtil {
 		}
 	}
 
-	public static void runOnMiniCluster(JobGraph jobGraph, int numberOfTaskTrackers) {
+	public static void runOnMiniCluster(JobGraph jobGraph, int numberOfTaskTrackers) throws Exception {
 		runOnMiniCluster(jobGraph, numberOfTaskTrackers, -1);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/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
index efb1e94..9586253 100644
--- 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
@@ -34,9 +34,7 @@ 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.streaming.util.ClusterUtil;
 import org.apache.flink.util.Collector;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class StreamComponentTest {
@@ -83,7 +81,6 @@ public class StreamComponentTest {
 	private static final int SOURCE_PARALELISM = 1;
 	private static final long MEMORYSIZE = 32;
 
-	@Ignore
 	@Test
 	public void wrongJobGraph() {
 		LocalStreamEnvironment env = StreamExecutionEnvironment
@@ -92,8 +89,7 @@ public class StreamComponentTest {
 		try {
 			env.execute();
 			fail();
-		} catch (RuntimeException e) {
-			assertEquals(e.getMessage(), ClusterUtil.CANNOT_EXECUTE_EMPTY_JOB);
+		} catch (Exception e) {
 		}
 
 		env.fromCollection(Arrays.asList("a", "b"));
@@ -101,8 +97,7 @@ public class StreamComponentTest {
 		try {
 			env.execute();
 			fail();
-		} catch (RuntimeException e) {
-			System.out.println(e.getMessage());
+		} catch (Exception e) {
 		}
 
 		try {
@@ -157,6 +152,7 @@ public class StreamComponentTest {
 	}
 
 	static HashSet<String> resultSet;
+
 	private static class SetSink implements SinkFunction<String> {
 		private static final long serialVersionUID = 1L;
 
@@ -165,33 +161,33 @@ public class StreamComponentTest {
 			resultSet.add(value);
 		}
 	}
-	
+
 	@Test
-	public void coTest() {
+	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"));
+
+		HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
+				"2", "3"));
 		assertEquals(expectedSet, resultSet);
 	}
 
 	@Test
-	public void runStream() {
+	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()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
index 7d8a49c..8f26be5 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/basictopology/BasicTopology.java
@@ -52,7 +52,7 @@ public class BasicTopology {
 	private static final int PARALLELISM = 1;
 	private static final int SOURCE_PARALLELISM = 1;
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment
 				.createLocalEnvironment(PARALLELISM);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
index 69b933c..5338e9d 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/cellinfo/CellInfoLocal.java
@@ -107,7 +107,7 @@ public class CellInfoLocal {
 	}
 
 	// Example for connecting data streams
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(
 				PARALLELISM).setBufferTimeout(100);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
index 472c8c3..05398f4 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/JoinLocal.java
@@ -30,7 +30,7 @@ public class JoinLocal {
 	// This example will join two streams. One which emits people's grades and
 	// one which emits people's salaries.
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(
 				PARALLELISM).setBufferTimeout(100);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index a433fd0..3218c47 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -117,7 +117,7 @@ public class IncrementalLearningSkeleton {
 	private static final int PARALLELISM = 1;
 	private static final int SOURCE_PARALLELISM = 1;
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(
 				PARALLELISM).setBufferTimeout(1000);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/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 b298f3f..cd8f71c 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
@@ -29,7 +29,7 @@ public class WindowJoinLocal {
 	// This example will join two streams with a sliding window. One which emits
 	// people's grades and one which emits people's salaries.
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(
 				PARALLELISM).setBufferTimeout(100);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/48d8ed70/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
index f78cd1a..532ec53 100644
--- 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
@@ -43,7 +43,7 @@ public class WordCountLocal {
 		}
 	}
 
-	public static void main(String[] args) {
+	public static void main(String[] args) throws Exception {
 
 		TestDataUtil.downloadIfNotExists("hamlet.txt");
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);


[02/18] git commit: [streaming] Added TypeInfo to DataStream

Posted by mb...@apache.org.
[streaming] Added TypeInfo to DataStream


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

Branch: refs/heads/master
Commit: 4d73f51c5acdabac98557fd53933d4b7ad42f98a
Parents: 0c8f1da
Author: mbalassi <ba...@gmail.com>
Authored: Fri Sep 5 14:41:53 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../api/datastream/ConnectedDataStream.java     |  21 +-
 .../streaming/api/datastream/DataStream.java    |  25 +-
 .../api/datastream/DataStreamSink.java          |   5 +-
 .../api/datastream/DataStreamSource.java        |   5 +-
 .../api/datastream/GroupedDataStream.java       |  26 +-
 .../api/datastream/IterativeDataStream.java     |   2 +-
 .../api/datastream/MergedDataStream.java        |   6 +-
 .../datastream/SingleOutputStreamOperator.java  | 292 ++++++++++---------
 .../api/datastream/SplitDataStream.java         |  14 +-
 .../environment/StreamExecutionEnvironment.java |  30 +-
 .../api/collector/DirectedOutputTest.java       |  19 +-
 .../api/invokable/operator/CoFlatMapTest.java   |  11 +-
 12 files changed, 270 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 4f17ceb..e00919f 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
@@ -35,6 +35,7 @@ import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.co.CoReduceInvokable;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * The ConnectedDataStream represents a stream for two different data types. It
@@ -58,7 +59,7 @@ public class ConnectedDataStream<IN1, IN2> {
 		this.jobGraphBuilder = jobGraphBuilder;
 		this.environment = environment;
 		this.input1 = input1.copy();
-		this.input2 = input2.copy();
+		this.input2 = input2.copy();		
 	}
 
 	/**
@@ -80,6 +81,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
 	 * applying function on grouped data streams for example
@@ -189,7 +206,7 @@ public class ConnectedDataStream<IN1, IN2> {
 
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(
-				environment, functionName);
+				environment, functionName, outTypeWrapper);
 
 		try {
 			input1.jobGraphBuilder.addCoTask(returnStream.getId(), functionInvokable,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 9375762..28d07d6 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
@@ -89,6 +89,7 @@ public abstract class DataStream<OUT> {
 	protected List<String> userDefinedNames;
 	protected boolean selectAll;
 	protected StreamPartitioner<OUT> partitioner;
+	protected TypeSerializerWrapper<OUT> outTypeWrapper;
 
 	protected final JobGraphBuilder jobGraphBuilder;
 
@@ -100,8 +101,11 @@ public abstract class DataStream<OUT> {
 	 *            StreamExecutionEnvironment
 	 * @param operatorType
 	 *            The type of the operator in the component
+	 * @param outTypeWrapper
+	 *            Type of the output
 	 */
-	public DataStream(StreamExecutionEnvironment environment, String operatorType) {
+	public DataStream(StreamExecutionEnvironment environment, String operatorType,
+			TypeSerializerWrapper<OUT> outTypeWrapper) {
 		if (environment == null) {
 			throw new NullPointerException("context is null");
 		}
@@ -114,7 +118,7 @@ public abstract class DataStream<OUT> {
 		this.userDefinedNames = new ArrayList<String>();
 		this.selectAll = false;
 		this.partitioner = new ForwardPartitioner<OUT>();
-
+		this.outTypeWrapper = outTypeWrapper;
 	}
 
 	/**
@@ -131,7 +135,7 @@ public abstract class DataStream<OUT> {
 		this.selectAll = dataStream.selectAll;
 		this.partitioner = dataStream.partitioner;
 		this.jobGraphBuilder = dataStream.jobGraphBuilder;
-
+		this.outTypeWrapper = dataStream.outTypeWrapper;
 	}
 
 	/**
@@ -160,6 +164,15 @@ public abstract class DataStream<OUT> {
 	}
 
 	/**
+	 * Gets the output type.
+	 * 
+	 * @return The output type.
+	 */
+	public TypeInformation<OUT> getOutputType() {
+		return this.outTypeWrapper.getTypeInfo();
+	}
+	
+	/**
 	 * Creates a new {@link MergedDataStream} by merging {@link DataStream}
 	 * outputs of the same type with each other. The DataStreams merged using
 	 * this operator will be transformed simultaneously.
@@ -890,7 +903,7 @@ public abstract class DataStream<OUT> {
 
 	protected <R> DataStream<OUT> addIterationSource(String iterationID, long waitTime) {
 
-		DataStream<R> returnStream = new DataStreamSource<R>(environment, "iterationSource");
+		DataStream<R> returnStream = new DataStreamSource<R>(environment, "iterationSource", null);
 
 		jobGraphBuilder.addIterationSource(returnStream.getId(), this.getId(), iterationID,
 				degreeOfParallelism, waitTime);
@@ -919,7 +932,7 @@ public abstract class DataStream<OUT> {
 		DataStream<OUT> inputStream = this.copy();
 		@SuppressWarnings({ "unchecked", "rawtypes" })
 		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment,
-				functionName);
+				functionName, outTypeWrapper);
 
 		try {
 			jobGraphBuilder.addTask(returnStream.getId(), functionInvokable, inTypeWrapper,
@@ -1001,7 +1014,7 @@ public abstract class DataStream<OUT> {
 
 	private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream,
 			SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT> typeWrapper) {
-		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink");
+		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink", outTypeWrapper);
 
 		try {
 			jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 71e88d8..f51e0e6 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,6 +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;
 
 /**
  * Represents the end of a DataStream.
@@ -27,8 +28,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  */
 public class DataStreamSink<IN> extends SingleOutputStreamOperator<IN, DataStreamSink<IN>> {
 
-	protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType) {
-		super(environment, operatorType);
+	protected DataStreamSink(StreamExecutionEnvironment environment, String operatorType, TypeSerializerWrapper<IN> outTypeWrapper) {
+		super(environment, operatorType, outTypeWrapper);
 	}
 
 	protected DataStreamSink(DataStream<IN> dataStream) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 2a05e27..29389ae 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,6 +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;
 
 /**
  * The DataStreamSource represents the starting point of a DataStream.
@@ -27,8 +28,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  */
 public class DataStreamSource<OUT> extends SingleOutputStreamOperator<OUT, DataStreamSource<OUT>> {
 
-	public DataStreamSource(StreamExecutionEnvironment environment, String operatorType) {
-		super(environment, operatorType);
+	public DataStreamSource(StreamExecutionEnvironment environment, String operatorType, TypeSerializerWrapper<OUT> outTypeWrapper) {
+		super(environment, operatorType, outTypeWrapper);
 	}
 
 	public DataStreamSource(DataStream<OUT> dataStream) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 94d6c8d..2e1ed57 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
@@ -31,8 +31,7 @@ import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvoka
 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.jasper.compiler.Node.ParamsAction;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * A GroupedDataStream represents a data stream which has been partitioned by
@@ -53,12 +52,21 @@ public class GroupedDataStream<OUT> {
 	}
 
 	/**
-	 * Applies a reduce transformation on the grouped data stream grouped by the
-	 * given key position. The {@link ReduceFunction} will receive input values
-	 * based on the key value. Only input values with the same key will go to
-	 * the same reducer.The user can also extend {@link RichReduceFunction} to
-	 * gain access to other features provided by the {@link RichFuntion}
-	 * interface.
+	 * Gets the output type.
+	 * 
+	 * @return The output type.
+	 */
+	public TypeInformation<OUT> getOutputType() {
+		return dataStream.getOutputType();
+	}
+	
+	/**
+	 * Applies a reduce transformation on the grouped data stream grouped on by
+	 * the given key position. The {@link ReduceFunction} will receive input
+	 * values based on the key value. Only input values with the same key will
+	 * go to the same reducer.The user can also extend
+	 * {@link RichReduceFunction} to gain access to other features provided by
+	 * the {@link RichFuntion} interface.
 	 * 
 	 * @param reducer
 	 *            The {@link ReduceFunction} that will be called for every
@@ -70,7 +78,7 @@ public class GroupedDataStream<OUT> {
 				ReduceFunction.class, 0), new FunctionTypeWrapper<OUT>(reducer,
 				ReduceFunction.class, 0), new GroupReduceInvokable<OUT>(reducer, keyPosition));
 	}
-
+	
 	/**
 	 * Applies a group reduce transformation on preset chunks of the grouped
 	 * data stream. The {@link GroupReduceFunction} will receive input values

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 16362ba..573dffd 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
@@ -80,7 +80,7 @@ public class IterativeDataStream<IN> extends
 	 * 
 	 */
 	public <R> DataStream<IN> closeWith(DataStream<IN> iterationTail, String iterationName) {
-		DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink");
+		DataStream<R> returnStream = new DataStreamSink<R>(environment, "iterationSink", null);
 
 		jobGraphBuilder.addIterationSink(returnStream.getId(), iterationTail.getId(),
 				iterationID.toString(), iterationTail.getParallelism(), waitTime);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
index 045af4f..c1618b2 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
 
 /**
  * The MergedDataStream represents a DataStream which consists of merged outputs
@@ -35,8 +36,9 @@ public class MergedDataStream<OUT> extends DataStream<OUT> {
 
 	protected List<DataStream<OUT>> mergedStreams;
 
-	protected MergedDataStream(StreamExecutionEnvironment environment, String operatorType) {
-		super(environment, operatorType);
+	protected MergedDataStream(StreamExecutionEnvironment environment, String operatorType,
+			TypeSerializerWrapper<OUT> outTypeWrapper) {
+		super(environment, operatorType, outTypeWrapper);
 		this.mergedStreams = new ArrayList<DataStream<OUT>>();
 		this.mergedStreams.add(this);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 1f01feb..b2fe551 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
@@ -1,147 +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
+ * 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.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;
-
-/**
- * The SingleOutputStreamOperator represents a user defined transformation
- * applied on a {@link DataStream} with one predefined output type.
- *
- * @param <OUT>
- *            Output type of the operator.
- * @param <O>
- *            Type of the operator.
- */
-public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends
-		DataStream<OUT> {
-
-	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, String operatorType) {
-		super(environment, operatorType);
-		setBufferTimeout(environment.getBufferTimeout());
-	}
-
-	protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
-		super(dataStream);
-	}
-
-	/**
-	 * Sets the degree of parallelism for this operator. The degree must be 1 or
-	 * more.
-	 * 
-	 * @param dop
-	 *            The degree of parallelism for this operator.
-	 * @return The operator with set degree of parallelism.
-	 */
-	public SingleOutputStreamOperator<OUT, O> setParallelism(int dop) {
-		if (dop < 1) {
-			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
-		}
-		this.degreeOfParallelism = dop;
-
-		jobGraphBuilder.setParallelism(id, degreeOfParallelism);
-
-		return this;
-	}
-
-	/**
-	 * Sets the mutability of the operator. If the operator is set to mutable,
-	 * the tuples received in the user defined functions, will be reused after
-	 * the function call. Setting an operator to mutable reduces garbage
-	 * collection overhead and thus increases scalability. Please note that if a
-	 * {@link DataStream#batchReduce} or {@link DataStream#windowReduce} is used
-	 * as mutable, the user can only iterate through the iterator once in every
-	 * invoke.
-	 * 
-	 * @param isMutable
-	 *            The mutability of the operator.
-	 * @return The operator with mutability set.
-	 */
-	public SingleOutputStreamOperator<OUT, O> setMutability(boolean isMutable) {
-		jobGraphBuilder.setMutability(id, isMutable);
-		return this;
-	}
-
-	/**
-	 * Sets the maximum time frequency (ms) for the flushing of the output
-	 * buffer. By default the output buffers flush only when they are full.
-	 * 
-	 * @param timeoutMillis
-	 *            The maximum time between two output flushes.
-	 * @return The operator with buffer timeout set.
-	 */
-	public SingleOutputStreamOperator<OUT, O> setBufferTimeout(long timeoutMillis) {
-		jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
-		return this;
-	}
-
-	/**
-	 * Operator used for directing tuples to specific named outputs using an
-	 * {@link OutputSelector}. Calling this method on an operator creates a new
-	 * {@link SplitDataStream}.
-	 * 
-	 * @param outputSelector
-	 *            The user defined {@link OutputSelector} for directing the
-	 *            tuples.
-	 * @return The {@link SplitDataStream}
-	 */
-	public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
-		try {
-			jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
-
-		} catch (SerializationException e) {
-			throw new RuntimeException("Cannot serialize OutputSelector");
-		}
-
-		return new SplitDataStream<OUT>(this);
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> partitionBy(int keyposition) {
-		return (SingleOutputStreamOperator<OUT, O>) super.partitionBy(keyposition);
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> broadcast() {
-		return (SingleOutputStreamOperator<OUT, O>) super.broadcast();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> shuffle() {
-		return (SingleOutputStreamOperator<OUT, O>) super.shuffle();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> forward() {
-		return (SingleOutputStreamOperator<OUT, O>) super.forward();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<OUT, O> distribute() {
-		return (SingleOutputStreamOperator<OUT, O>) super.distribute();
-	}
-
-	@Override
-	protected SingleOutputStreamOperator<OUT, O> copy() {
-		return new SingleOutputStreamOperator<OUT, O>(this);
-	}
-
-}
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+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;
+
+/**
+ * The SingleOutputStreamOperator represents a user defined transformation
+ * applied on a {@link DataStream} with one predefined output type.
+ *
+ * @param <OUT>
+ *            Output type of the operator.
+ * @param <O>
+ *            Type of the operator.
+ */
+public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperator<OUT, O>> extends
+		DataStream<OUT> {
+
+	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment,
+			String operatorType, TypeSerializerWrapper<OUT> outTypeWrapper) {
+		super(environment, operatorType, outTypeWrapper);
+		setBufferTimeout(environment.getBufferTimeout());
+	}
+
+	protected SingleOutputStreamOperator(DataStream<OUT> dataStream) {
+		super(dataStream);
+	}
+
+	/**
+	 * Sets the degree of parallelism for this operator. The degree must be 1 or
+	 * more.
+	 * 
+	 * @param dop
+	 *            The degree of parallelism for this operator.
+	 * @return The operator with set degree of parallelism.
+	 */
+	public SingleOutputStreamOperator<OUT, O> setParallelism(int dop) {
+		if (dop < 1) {
+			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
+		}
+		this.degreeOfParallelism = dop;
+
+		jobGraphBuilder.setParallelism(id, degreeOfParallelism);
+
+		return this;
+	}
+
+	/**
+	 * Sets the mutability of the operator. If the operator is set to mutable,
+	 * the tuples received in the user defined functions, will be reused after
+	 * the function call. Setting an operator to mutable reduces garbage
+	 * collection overhead and thus increases scalability. Please note that if a
+	 * {@link DataStream#batchReduce} or {@link DataStream#windowReduce} is used
+	 * as mutable, the user can only iterate through the iterator once in every
+	 * invoke.
+	 * 
+	 * @param isMutable
+	 *            The mutability of the operator.
+	 * @return The operator with mutability set.
+	 */
+	public SingleOutputStreamOperator<OUT, O> setMutability(boolean isMutable) {
+		jobGraphBuilder.setMutability(id, isMutable);
+		return this;
+	}
+
+	/**
+	 * Sets the maximum time frequency (ms) for the flushing of the output
+	 * buffer. By default the output buffers flush only when they are full.
+	 * 
+	 * @param timeoutMillis
+	 *            The maximum time between two output flushes.
+	 * @return The operator with buffer timeout set.
+	 */
+	public SingleOutputStreamOperator<OUT, O> setBufferTimeout(long timeoutMillis) {
+		jobGraphBuilder.setBufferTimeout(id, timeoutMillis);
+		return this;
+	}
+
+	/**
+	 * Operator used for directing tuples to specific named outputs using an
+	 * {@link OutputSelector}. Calling this method on an operator creates a new
+	 * {@link SplitDataStream}.
+	 * 
+	 * @param outputSelector
+	 *            The user defined {@link OutputSelector} for directing the
+	 *            tuples.
+	 * @return The {@link SplitDataStream}
+	 */
+	public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
+		try {
+			jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
+
+		} catch (SerializationException e) {
+			throw new RuntimeException("Cannot serialize OutputSelector");
+		}
+
+		return new SplitDataStream<OUT>(this);
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, O> partitionBy(int keyposition) {
+		return (SingleOutputStreamOperator<OUT, O>) super.partitionBy(keyposition);
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, O> broadcast() {
+		return (SingleOutputStreamOperator<OUT, O>) super.broadcast();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, O> shuffle() {
+		return (SingleOutputStreamOperator<OUT, O>) super.shuffle();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, O> forward() {
+		return (SingleOutputStreamOperator<OUT, O>) super.forward();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, O> distribute() {
+		return (SingleOutputStreamOperator<OUT, O>) super.distribute();
+	}
+
+	@Override
+	protected SingleOutputStreamOperator<OUT, O> copy() {
+		return new SingleOutputStreamOperator<OUT, O>(this);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
index 0ddb4f0..838f228 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
@@ -18,8 +18,9 @@
 package org.apache.flink.streaming.api.datastream;
 
 import java.util.Arrays;
-
+
 import org.apache.flink.streaming.api.collector.OutputSelector;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * The SplitDataStream represents an operator that has been split using an
@@ -36,7 +37,16 @@ public class SplitDataStream<OUT> {
 	protected SplitDataStream(DataStream<OUT> dataStream) {
 		this.dataStream = dataStream.copy();
 	}
-
+
+	/**
+	 * Gets the output type.
+	 * 
+	 * @return The output type.
+	 */
+	public TypeInformation<OUT> getOutputType() {
+		return dataStream.getOutputType();
+	}
+	
 	/**
 	 * Sets the output names for which the next operator will receive values.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/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 7d983ad..6187c99 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,6 +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;
 
 /**
  * {@link ExecutionEnvironment} for streaming jobs. An instance of it is
@@ -189,7 +190,6 @@ public abstract class StreamExecutionEnvironment {
 		return addSource(new FileStreamFunction(filePath), parallelism);
 	}
 
-
 	private static void checkIfFileExists(String filePath) {
 		File file = new File(filePath);
 		if (!file.exists()) {
@@ -199,12 +199,12 @@ public abstract class StreamExecutionEnvironment {
 		if (!file.canRead()) {
 			throw new IllegalArgumentException("Cannot read file: " + filePath);
 		}
-		
+
 		if (file.isDirectory()) {
 			throw new IllegalArgumentException("Given path is a directory: " + filePath);
 		}
 	}
-	
+
 	/**
 	 * Creates a new DataStream that contains the given elements. The elements
 	 * must all be of the same type, for example, all of the String or Integer.
@@ -219,18 +219,19 @@ public abstract class StreamExecutionEnvironment {
 	 * @return The DataStream representing the elements.
 	 */
 	public <OUT extends Serializable> DataStreamSource<OUT> fromElements(OUT... data) {
-		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
-
 		if (data.length == 0) {
 			throw new IllegalArgumentException(
 					"fromElements needs at least one element as argument");
 		}
 
+		TypeSerializerWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data[0]);
+		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements",
+				outTypeWrapper);
+
 		try {
 			SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
 			jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
-					new ObjectTypeWrapper<OUT>(data[0]), "source",
-					SerializationUtils.serialize(function), 1);
+					outTypeWrapper, "source", SerializationUtils.serialize(function), 1);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize elements");
 		}
@@ -250,8 +251,6 @@ public abstract class StreamExecutionEnvironment {
 	 * @return The DataStream representing the elements.
 	 */
 	public <OUT extends Serializable> DataStreamSource<OUT> fromCollection(Collection<OUT> data) {
-		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements");
-
 		if (data == null) {
 			throw new NullPointerException("Collection must not be null");
 		}
@@ -260,6 +259,11 @@ public abstract class StreamExecutionEnvironment {
 			throw new IllegalArgumentException("Collection must not be empty");
 		}
 
+		TypeSerializerWrapper<OUT> outTypeWrapper = new ObjectTypeWrapper<OUT>(data.iterator()
+				.next());
+		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "elements",
+				outTypeWrapper);
+
 		try {
 			SourceFunction<OUT> function = new FromElementsFunction<OUT>(data);
 
@@ -301,12 +305,14 @@ public abstract class StreamExecutionEnvironment {
 	 * @return the data stream constructed
 	 */
 	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, int parallelism) {
-		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "source");
+		TypeSerializerWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(function,
+				SourceFunction.class, 0);
+		DataStreamSource<OUT> returnStream = new DataStreamSource<OUT>(this, "source",
+				outTypeWrapper);
 
 		try {
 			jobGraphBuilder.addSource(returnStream.getId(), new SourceInvokable<OUT>(function),
-					new FunctionTypeWrapper<OUT>(function, SourceFunction.class, 0), "source",
-					SerializationUtils.serialize(function), parallelism);
+					outTypeWrapper, "source", SerializationUtils.serialize(function), parallelism);
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot serialize SourceFunction");
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index fdf9db3..e9d3994 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -1,3 +1,20 @@
+/**
+ * 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.collector;
 
 import static org.junit.Assert.assertEquals;
@@ -82,7 +99,7 @@ public class DirectedOutputTest {
 	public void outputSelectorTest() throws Exception {
 
 		LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
+		
 		SplitDataStream<Long> source = env.generateSequence(1, 10).split(new MyOutputSelector());
 		source.select(EVEN).addSink(new ListSink(EVEN));
 		source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4d73f51c/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
index e7f12bd..0c4bad1 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
@@ -72,12 +72,19 @@ public class CoFlatMapTest implements Serializable {
 
 		DataStream<Integer> ds1 = env.fromElements(1, 3, 5);
 		DataStream<Integer> ds2 = env.fromElements(2, 4).merge(ds1);
-
+		
 		try {
 			ds1.forward().merge(ds2);
 			fail();
 		} catch (RuntimeException e) {
-			// good
+			// expected
+		}
+		
+		try {
+			env.fromElements(10, 11).connect(ds2);
+			fail();
+		} catch (RuntimeException e) {
+			// expected
 		}
 	}
 }


[06/18] git commit: [streaming] window and batch operator added to DataStream + Documentation updated accordingly

Posted by mb...@apache.org.
[streaming] window and batch operator added to DataStream + Documentation updated accordingly


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

Branch: refs/heads/master
Commit: 5f601cf9b18fef0b54a92e42405c0179e639f5da
Parents: 47d02a0
Author: gyfora <gy...@gmail.com>
Authored: Mon Sep 8 02:05:20 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md                         |  50 ++--
 .../api/datastream/BatchedDataStream.java       | 238 +++++++++++++++++++
 .../streaming/api/datastream/DataStream.java    | 197 ++++++---------
 .../api/datastream/GroupedDataStream.java       |   8 +-
 .../api/datastream/WindowDataStream.java        |  87 +++++++
 .../GroupedWindowGroupReduceInvokable.java      |   4 +-
 .../operator/GroupedWindowReduceInvokable.java  |  19 +-
 .../operator/WindowGroupReduceInvokable.java    |  12 +-
 .../operator/WindowReduceInvokable.java         |  18 +-
 .../api/invokable/util/DefaultTimeStamp.java    |  39 +++
 .../api/invokable/util/DefaultTimestamp.java    |  34 ---
 .../streaming/api/invokable/util/TimeStamp.java |  46 ++++
 .../streaming/api/invokable/util/Timestamp.java |  38 ---
 .../WindowGroupReduceInvokableTest.java         |  19 +-
 .../operator/WindowReduceInvokableTest.java     |  18 +-
 .../ml/IncrementalLearningSkeleton.java         |   6 +-
 16 files changed, 561 insertions(+), 272 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index 04e2f2e..6ed53df 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -242,42 +242,42 @@ Merges two or more `DataStream` instances creating a new DataStream containing a
 dataStream.merge(otherStream1, otherStream2…)
 ```
 
+### Grouped operators
+
+Some transformations require that the `DataStream` is grouped on some key value. The user can create a `GroupedDataStream` by calling the `groupBy(keyPosition)` method of a non-grouped `DataStream`. The user can apply different reduce transformations on the obtained `GroupedDataStream`:
+
+#### Reduce on GroupedDataStream
+When the reduce operator is applied on a grouped data stream, the user-defined `ReduceFunction` will combine subsequent pairs of elements having the same key value. The combined results are sent to the output stream.
+
+### Aggregations
+
+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)`
+
+For every incoming tuple the selected field is replaced with the current aggregated value. If the aggregations are used without defining field position, 0 is used as default. 
+
 ### Window/Batch operators
 
 Window and batch operators allow the user to execute function on slices or windows of the DataStream in a sliding fashion. If the stepsize for the slide is not defined then the window/batchsize is used as stepsize by default.
 
-#### Window reduce
-The transformation calls a user-defined `GroupReduceFunction` on records received during the predefined time window. The window is shifted after each reduce call.
-A window reduce that sums the elements in the last minute with 10 seconds stepsize:
+When applied to grouped data streams the operators applied will be executed on groups of elements grouped by the selected key position.
 
-```java
-dataStream.windowReduce(new GroupReduceFunction<Integer, Integer>() {
-			@Override
-			public void reduce(Iterable<Integer> values, Collector<Integer> out) throws Exception {
-				Integer sum = 0;
-				for(Integer val: values){
-					sum+=val;
-				}
-			}
-		}, 60000, 10000);
-```
+#### Reduce on windowed/batched data streams
+The transformation calls a user-defined `ReduceFunction` on records received in the batch or during the predefined time window. The window is shifted after each reduce call. The user can also use the different streaming aggregations.
 
-#### Batch reduce
-The transformation calls a `GroupReduceFunction` for each data batch of the predefined size. The batch slides by the predefined number of elements after each call. Works similarly to window reduce.
+A window reduce that sums the elements in the last minute with 10 seconds slide interval:
 
 ```java
-dataStream.batchReduce(reducer, batchSize, slideSize)
+dataStream.window(60000, 10000).sum();
 ```
 
-### Grouped operators
-
-Some transformations require that the `DataStream` is grouped on some key value. The user can create a `GroupedDataStream` by calling the `groupBy(keyPosition)` method of a non-grouped `DataStream`. The user can apply different reduce transformations on the obtained `GroupedDataStream`:
-
-#### Reduce on GroupedDataStream
-When the reduce operator is applied on a grouped data stream, the user-defined `ReduceFunction` will combine subsequent pairs of elements having the same key value. The combined results are sent to the output stream.
+#### ReduceGroup on windowed/batched data streams
+The transformation calls a `GroupReduceFunction` for each data batch or data window. The batch/window slides by the predefined number of elements/time after each call.
 
-#### Window/Batchreduce on GroupedDataStream
-Similarly to the grouped reduce operator the window and batch reduce operators work the same way as in the non-grouped case except that in a data window/batch every `GroupReduceFunction` call will receive data elements for only the same keys.
+```java
+dataStream.batch(1000, 100).reduceGroup(reducer);
+```
 
 ### Co operators
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/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
new file mode 100755
index 0000000..0aa5de6
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/BatchedDataStream.java
@@ -0,0 +1,238 @@
+/**
+ * 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.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.functions.RichReduceFunction;
+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.operator.BatchGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedBatchReduceInvokable;
+import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
+import org.apache.flink.types.TypeInformation;
+
+/**
+ * A {@link BatchedDataStream} represents a data stream whose elements are
+ * batched together in a sliding batch. operations like
+ * {@link #reduce(ReduceFunction)} or {@link #reduceGroup(GroupReduceFunction)}
+ * are applied for each batch and the batch is slid afterwards.
+ *
+ * @param <OUT>
+ *            The output type of the {@link BatchedDataStream}
+ */
+public class BatchedDataStream<OUT> {
+
+	protected DataStream<OUT> dataStream;
+	protected boolean isGrouped;
+	protected int keyPosition;
+	protected long batchSize;
+	protected long slideSize;
+
+	protected BatchedDataStream(DataStream<OUT> dataStream, long batchSize, long slideSize) {
+		if (dataStream instanceof GroupedDataStream) {
+			this.isGrouped = true;
+			this.keyPosition = ((GroupedDataStream<OUT>) dataStream).keyPosition;
+		} else {
+			this.isGrouped = false;
+		}
+		this.dataStream = dataStream.copy();
+		this.batchSize = batchSize;
+		this.slideSize = slideSize;
+	}
+
+	protected BatchedDataStream(BatchedDataStream<OUT> batchedDataStream) {
+		this.dataStream = batchedDataStream.dataStream.copy();
+		this.isGrouped = batchedDataStream.isGrouped;
+		this.keyPosition = batchedDataStream.keyPosition;
+		this.batchSize = batchedDataStream.batchSize;
+		this.slideSize = batchedDataStream.slideSize;
+	}
+
+	/**
+	 * Groups the elements of the {@link BatchedDataStream} by the given key
+	 * position to be used with grouped operators.
+	 * 
+	 * @param keyPosition
+	 *            The position of the field on which the
+	 *            {@link BatchedDataStream} will be grouped.
+	 * @return The transformed {@link BatchedDataStream}
+	 */
+	public BatchedDataStream<OUT> groupBy(int keyPosition) {
+		return new BatchedDataStream<OUT>(dataStream.groupBy(keyPosition), batchSize, slideSize);
+	}
+
+	/**
+	 * Applies a reduce transformation on every sliding batch/window of the data
+	 * stream. If the data stream is grouped then the reducer is applied on
+	 * every group of elements sharing the same key. This type of reduce is much
+	 * faster than reduceGroup since the reduce function can be applied
+	 * incrementally. The user can also extend the {@link RichReduceFunction} to
+	 * gain access to other features provided by the {@link RichFuntion}
+	 * interface.
+	 * 
+	 * @param reducer
+	 *            The {@link ReduceFunction} that will be called for every
+	 *            element of the input values in the batch/window.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> reduce(ReduceFunction<OUT> reducer) {
+		return dataStream.addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
+				ReduceFunction.class, 0), new FunctionTypeWrapper<OUT>(reducer,
+				ReduceFunction.class, 0), getReduceInvokable(reducer));
+	}
+
+	/**
+	 * Applies a reduceGroup transformation on preset batches/windows of the
+	 * DataStream. The transformation calls a {@link GroupReduceFunction} for
+	 * each batch/window. Each GroupReduceFunction call can return any number of
+	 * elements including none. The user can also extend
+	 * {@link RichGroupReduceFunction} to gain access to other features provided
+	 * by the {@link RichFuntion} interface.
+	 * 
+	 * @param reducer
+	 *            The {@link GroupReduceFunction} that will be called for every
+	 *            batch/window.
+	 * @return The transformed DataStream.
+	 */
+	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));
+	}
+
+	/**
+	 * Applies an aggregation that sums every sliding batch/window of the data
+	 * stream at the given position.
+	 * 
+	 * @param positionToSum
+	 *            The position in the data point to sum
+	 * @return The transformed DataStream.
+	 */
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<OUT, ?> sum(int positionToSum) {
+		dataStream.checkFieldRange(positionToSum);
+		return aggregate((AggregationFunction<OUT>) SumAggregationFunction.getSumFunction(
+				positionToSum, dataStream.getClassAtPos(positionToSum)));
+	}
+
+	/**
+	 * Syntactic sugar for sum(0)
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> sum() {
+		return sum(0);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum of every sliding
+	 * batch/window of the data stream at the given position.
+	 * 
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min(int positionToMin) {
+		dataStream.checkFieldRange(positionToMin);
+		return aggregate(new MinAggregationFunction<OUT>(positionToMin));
+	}
+
+	/**
+	 * Syntactic sugar for min(0)
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min() {
+		return min(0);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum of every sliding
+	 * batch/window of the data stream at the given position.
+	 * 
+	 * @param positionToMax
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max(int positionToMax) {
+		dataStream.checkFieldRange(positionToMax);
+		return aggregate(new MaxAggregationFunction<OUT>(positionToMax));
+	}
+
+	/**
+	 * Syntactic sugar for max(0)
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max() {
+		return max(0);
+	}
+
+	/**
+	 * Gets the output type.
+	 * 
+	 * @return The output type.
+	 */
+	public TypeInformation<OUT> getOutputType() {
+		return dataStream.getOutputType();
+	}
+
+	private SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
+		BatchReduceInvokable<OUT> invokable = getReduceInvokable(aggregate);
+
+		SingleOutputStreamOperator<OUT, ?> returnStream = dataStream.addFunction("batchReduce",
+				aggregate, null, null, invokable);
+
+		dataStream.jobGraphBuilder.setTypeWrappersFrom(dataStream.getId(), returnStream.getId());
+		return returnStream;
+	}
+
+	protected BatchReduceInvokable<OUT> getReduceInvokable(ReduceFunction<OUT> reducer) {
+		BatchReduceInvokable<OUT> invokable;
+		if (isGrouped) {
+			invokable = new GroupedBatchReduceInvokable<OUT>(reducer, batchSize, slideSize,
+					keyPosition);
+		} else {
+			invokable = new BatchReduceInvokable<OUT>(reducer, batchSize, slideSize);
+		}
+		return invokable;
+	}
+
+	protected <R> BatchGroupReduceInvokable<OUT, R> getGroupReduceInvokable(
+			GroupReduceFunction<OUT, R> reducer) {
+		BatchGroupReduceInvokable<OUT, R> invokable;
+		if (isGrouped) {
+			invokable = new GroupedBatchGroupReduceInvokable<OUT, R>(reducer, batchSize, slideSize,
+					keyPosition);
+		} else {
+			invokable = new BatchGroupReduceInvokable<OUT, R>(reducer, batchSize, slideSize);
+		}
+		return invokable;
+	}
+
+	protected BatchedDataStream<OUT> copy() {
+		return new BatchedDataStream<OUT>(this);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/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 70348d6..bebda91 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
@@ -26,12 +26,10 @@ import org.apache.commons.lang3.SerializationUtils;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.functions.RichFilterFunction;
 import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichMapFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
@@ -49,14 +47,12 @@ 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.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
-import org.apache.flink.streaming.api.invokable.util.DefaultTimestamp;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.partitioner.DistributePartitioner;
 import org.apache.flink.streaming.partitioner.FieldsPartitioner;
@@ -385,160 +381,113 @@ public class DataStream<OUT> {
 				ReduceFunction.class, 0), new StreamReduceInvokable<OUT>(reducer));
 	}
 
-	public GroupedDataStream<OUT> groupBy(int keyPosition) {
-		return new GroupedDataStream<OUT>(this, keyPosition);
-	}
-
 	/**
-	 * Applies a reduce transformation on preset chunks of the DataStream. The
-	 * transformation calls a {@link GroupReduceFunction} for each tuple batch
-	 * of the predefined size. Each GroupReduceFunction call can return any
-	 * number of elements including none. The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
+	 * Groups the elements of a {@link DataStream} by the given key position to
+	 * be used with grouped operators like
+	 * {@link GroupedDataStream#reduce(ReduceFunction)}
 	 * 
-	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each tuple batch.
-	 * @param batchSize
-	 *            The number of tuples grouped together in the batch.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed {@link DataStream}.
+	 * @param keyPosition
+	 *            The position of the field on which the {@link DataStream} will
+	 *            be grouped.
+	 * @return The transformed {@link DataStream}
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
-			long batchSize) {
-		return batchReduce(reducer, batchSize, batchSize);
+	public GroupedDataStream<OUT> groupBy(int keyPosition) {
+		return new GroupedDataStream<OUT>(this, keyPosition);
 	}
 
 	/**
-	 * Applies a reduce transformation on preset sliding chunks of the
-	 * DataStream. The transformation calls a {@link GroupReduceFunction} for
-	 * each tuple batch of the predefined size. The tuple batch gets slid by the
-	 * given number of tuples. Each GroupReduceFunction call can return any
-	 * number of elements including none. The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
-	 * 
+	 * Collects the data stream elements into sliding batches creating a new
+	 * {@link BatchedDataStream}. The user can apply transformations like
+	 * {@link BatchedDataStream#reduce}, {@link BatchedDataStream#reduceGroup}
+	 * or aggregations on the {@link BatchedDataStream}.
 	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each tuple batch.
 	 * @param batchSize
-	 *            The number of tuples grouped together in the batch.
+	 *            The number of elements in each batch at each operator
 	 * @param slideSize
-	 *            The number of tuples the batch is slid by.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed {@link DataStream}.
+	 *            The number of elements with which the batches are slid by
+	 *            after each transformation.
+	 * @return The transformed {@link DataStream}
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
-			long batchSize, long slideSize) {
+	public BatchedDataStream<OUT> batch(long batchSize, long slideSize) {
 		if (batchSize < 1) {
 			throw new IllegalArgumentException("Batch size must be positive");
 		}
 		if (slideSize < 1) {
 			throw new IllegalArgumentException("Slide size must be positive");
 		}
-
-		FunctionTypeWrapper<OUT> inTypeWrapper = new FunctionTypeWrapper<OUT>(reducer,
-				GroupReduceFunction.class, 0);
-		FunctionTypeWrapper<R> outTypeWrapper = new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1);
-
-		return addFunction("batchReduce", reducer, inTypeWrapper, outTypeWrapper,
-				new BatchGroupReduceInvokable<OUT, R>(reducer, batchSize, slideSize));
+		return new BatchedDataStream<OUT>(this, batchSize, slideSize);
 	}
 
 	/**
-	 * Applies a reduce transformation on preset "time" chunks of the
-	 * DataStream. The transformation calls a {@link GroupReduceFunction} on
-	 * records received during the predefined time window. The window is shifted
-	 * after each reduce call. Each GroupReduceFunction call can return any
-	 * number of elements including none.The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
-	 * 
+	 * Collects the data stream elements into sliding batches creating a new
+	 * {@link BatchedDataStream}. The user can apply transformations like
+	 * {@link BatchedDataStream#reduce}, {@link BatchedDataStream#reduceGroup}
+	 * or aggregations on the {@link BatchedDataStream}.
 	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
-	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize) {
-		return windowReduce(reducer, windowSize, windowSize);
-	}
-
-	/**
-	 * Applies a reduce transformation on preset "time" chunks of the
-	 * DataStream. The transformation calls a {@link GroupReduceFunction} on
-	 * records received during the predefined time window. The window is shifted
-	 * after each reduce call. Each GroupReduceFunction call can return any
-	 * number of elements including none.The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
-	 * 
-	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
-	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
-	 * @param slideInterval
-	 *            The time interval, batch is slid by.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed DataStream.
+	 * @param batchSize
+	 *            The number of elements in each batch at each operator
+	 * @return The transformed {@link DataStream}
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize, long slideInterval) {
-		return windowReduce(reducer, windowSize, slideInterval, new DefaultTimestamp<OUT>());
+	public BatchedDataStream<OUT> batch(long batchSize) {
+		return batch(batchSize, batchSize);
 	}
 
 	/**
-	 * Applies a reduce transformation on preset "time" chunks of the
-	 * DataStream. The transformation calls a {@link GroupReduceFunction} on
-	 * records received during the predefined time window. The window is shifted
-	 * after each reduce call. Each GroupReduceFunction call can return any
-	 * number of elements including none. The time is determined by a
-	 * user-defined timestamp. The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
+	 * Collects the data stream elements into sliding windows creating a new
+	 * {@link WindowDataStream}. The user can apply transformations like
+	 * {@link WindowDataStream#reduce}, {@link WindowDataStream#reduceGroup} or
+	 * aggregations on the {@link WindowDataStream}.
 	 * 
-	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
 	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
+	 *            The length of the window in milliseconds.
 	 * @param slideInterval
-	 *            The time interval, batch is slid by.
+	 *            The number of milliseconds with which the windows are slid by
+	 *            after each transformation.
 	 * @param timestamp
-	 *            Timestamp function to retrieve a timestamp from an element.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed DataStream.
+	 *            User defined function for extracting time-stamps from each
+	 *            element
+	 * @return The transformed {@link DataStream}
 	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize, long slideInterval, Timestamp<OUT> timestamp) {
+	public WindowDataStream<OUT> window(long windowSize, long slideInterval,
+			TimeStamp<OUT> timestamp) {
 		if (windowSize < 1) {
 			throw new IllegalArgumentException("Window size must be positive");
 		}
 		if (slideInterval < 1) {
 			throw new IllegalArgumentException("Slide interval must be positive");
 		}
+		return new WindowDataStream<OUT>(this, windowSize, slideInterval, timestamp);
+	}
 
-		FunctionTypeWrapper<OUT> inTypeWrapper = new FunctionTypeWrapper<OUT>(reducer,
-				GroupReduceFunction.class, 0);
-		FunctionTypeWrapper<R> outTypeWrapper = new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1);
+	/**
+	 * Collects the data stream elements into sliding windows creating a new
+	 * {@link WindowDataStream}. The user can apply transformations like
+	 * {@link WindowDataStream#reduce}, {@link WindowDataStream#reduceGroup} or
+	 * aggregations on the {@link WindowDataStream}.
+	 * 
+	 * @param windowSize
+	 *            The length of the window in milliseconds.
+	 * @param slideInterval
+	 *            The number of milliseconds with which the windows are slid by
+	 *            after each transformation.
+	 * @return The transformed {@link DataStream}
+	 */
+	public WindowDataStream<OUT> window(long windowSize, long slideInterval) {
+		return window(windowSize, slideInterval, new DefaultTimeStamp<OUT>());
+	}
 
-		return addFunction("batchReduce", reducer, inTypeWrapper, outTypeWrapper,
-				new WindowGroupReduceInvokable<OUT, R>(reducer, windowSize, slideInterval, timestamp));
+	/**
+	 * Collects the data stream elements into sliding windows creating a new
+	 * {@link WindowDataStream}. The user can apply transformations like
+	 * {@link WindowDataStream#reduce}, {@link WindowDataStream#reduceGroup} or
+	 * aggregations on the {@link WindowDataStream}.
+	 * 
+	 * @param windowSize
+	 *            The length of the window in milliseconds.
+	 * @return The transformed {@link DataStream}
+	 */
+	public WindowDataStream<OUT> window(long windowSize) {
+		return window(windowSize, windowSize);
 	}
 
 	/**
@@ -1115,7 +1064,7 @@ public class DataStream<OUT> {
 	 * 
 	 * @return The copy
 	 */
-	protected DataStream<OUT> copy(){
+	protected DataStream<OUT> copy() {
 		return new DataStream<OUT>(this);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/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 e513f2d..138a6f8 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
@@ -25,8 +25,8 @@ import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupedWindowGroupReduceInvokable;
-import org.apache.flink.streaming.api.invokable.util.DefaultTimestamp;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 
@@ -166,7 +166,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	 */
 	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
 			long windowSize, long slideInterval) {
-		return windowReduce(reducer, windowSize, slideInterval, new DefaultTimestamp<OUT>());
+		return windowReduce(reducer, windowSize, slideInterval, new DefaultTimeStamp<OUT>());
 	}
 
 	/**
@@ -191,7 +191,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize, long slideInterval, Timestamp<OUT> timestamp) {
+			long windowSize, long slideInterval, TimeStamp<OUT> timestamp) {
 		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
 				GroupReduceFunction.class, 1), new GroupedWindowGroupReduceInvokable<OUT, R>(reducer,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
new file mode 100755
index 0000000..4756050
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
@@ -0,0 +1,87 @@
+/**
+ * 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.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.GroupedWindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedWindowReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
+
+/**
+ * A {@link WindowDataStream} represents a data stream whose elements are
+ * batched together in a sliding window. operations like
+ * {@link #reduce(ReduceFunction)} or {@link #reduceGroup(GroupReduceFunction)}
+ * are applied for each window and the window is slid afterwards.
+ *
+ * @param <OUT>
+ *            The output type of the {@link WindowDataStream}
+ */
+public class WindowDataStream<OUT> extends BatchedDataStream<OUT> {
+
+	TimeStamp<OUT> timeStamp;
+
+	protected WindowDataStream(DataStream<OUT> dataStream, long windowSize, long slideInterval,
+			TimeStamp<OUT> timeStamp) {
+		super(dataStream, windowSize, slideInterval);
+		this.timeStamp = timeStamp;
+	}
+
+	protected WindowDataStream(WindowDataStream<OUT> windowDataStream) {
+		super(windowDataStream);
+		this.timeStamp = windowDataStream.timeStamp;
+	}
+
+	public WindowDataStream<OUT> groupBy(int keyPosition) {
+		return new WindowDataStream<OUT>(dataStream.groupBy(keyPosition), batchSize, slideSize,
+				timeStamp);
+	}
+
+	protected BatchReduceInvokable<OUT> getReduceInvokable(ReduceFunction<OUT> reducer) {
+		BatchReduceInvokable<OUT> invokable;
+		if (isGrouped) {
+			invokable = new GroupedWindowReduceInvokable<OUT>(reducer, batchSize, slideSize,
+					keyPosition, timeStamp);
+		} else {
+			invokable = new WindowReduceInvokable<OUT>(reducer, batchSize, slideSize, timeStamp);
+		}
+		return invokable;
+	}
+
+	protected <R> BatchGroupReduceInvokable<OUT, R> getGroupReduceInvokable(
+			GroupReduceFunction<OUT, R> reducer) {
+		BatchGroupReduceInvokable<OUT, R> invokable;
+		if (isGrouped) {
+			invokable = new GroupedWindowGroupReduceInvokable<OUT, R>(reducer, batchSize,
+					slideSize, keyPosition, timeStamp);
+		} else {
+			invokable = new WindowGroupReduceInvokable<OUT, R>(reducer, batchSize, slideSize,
+					timeStamp);
+		}
+		return invokable;
+	}
+
+	public WindowDataStream<OUT> copy() {
+		return new WindowDataStream<OUT>(this);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
index 4027b78..865dced 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
@@ -22,7 +22,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 import org.apache.flink.streaming.state.MutableTableState;
 
@@ -33,7 +33,7 @@ public class GroupedWindowGroupReduceInvokable<IN, OUT> extends WindowGroupReduc
 	private MutableTableState<Object, List<IN>> values;
 
 	public GroupedWindowGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
-			long slideInterval, int keyPosition, Timestamp<IN> timestamp) {
+			long slideInterval, int keyPosition, TimeStamp<IN> timestamp) {
 		super(reduceFunction, windowSize, slideInterval, timestamp);
 		this.keyPosition = keyPosition;
 		this.reducer = reduceFunction;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
index e202e86..df94843 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
@@ -18,33 +18,26 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import java.io.IOException;
-import java.util.Map;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.state.SlidingWindowState;
 
 public class GroupedWindowReduceInvokable<OUT> extends GroupedBatchReduceInvokable<OUT> {
 
 	private static final long serialVersionUID = 1L;
-	protected transient SlidingWindowState<Map<Object, OUT>> state;
 
-	private Timestamp<OUT> timestamp;
+	private TimeStamp<OUT> timestamp;
 	private long startTime;
 	private long nextRecordTime;
 
 	public GroupedWindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
-			long slideInterval, Timestamp<OUT> timestamp, int keyPosition) {
+			long slideInterval, int keyPosition, TimeStamp<OUT> timestamp) {
 		super(reduceFunction, windowSize, slideInterval, keyPosition);
 		this.timestamp = timestamp;
+		this.startTime = timestamp.getStartTime();
 	}
-	
-	@Override
-	protected void initializeAtFirstRecord() {
-		startTime = nextRecordTime - (nextRecordTime % granularity);
-	}
-	
+
 	@Override
 	protected StreamRecord<OUT> getNextRecord() throws IOException {
 		reuse = recordIterator.next(reuse);
@@ -53,7 +46,7 @@ public class GroupedWindowReduceInvokable<OUT> extends GroupedBatchReduceInvokab
 		}
 		return reuse;
 	}
-	
+
 	@Override
 	protected boolean batchNotFull() {
 		if (nextRecordTime < startTime + granularity) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
index 7b4317a..03c19d4 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
@@ -20,24 +20,20 @@ package org.apache.flink.streaming.api.invokable.operator;
 import java.io.IOException;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
 public class WindowGroupReduceInvokable<IN, OUT> extends BatchGroupReduceInvokable<IN, OUT> {
 	private static final long serialVersionUID = 1L;
 	private long startTime;
 	private long nextRecordTime;
-	private Timestamp<IN> timestamp;
+	private TimeStamp<IN> timestamp;
 
 	public WindowGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
-			long slideInterval, Timestamp<IN> timestamp) {
+			long slideInterval, TimeStamp<IN> timestamp) {
 		super(reduceFunction, windowSize, slideInterval);
 		this.timestamp = timestamp;
-	}
-
-	@Override
-	protected void initializeAtFirstRecord() {
-		startTime = nextRecordTime - (nextRecordTime % granularity);
+		this.startTime = timestamp.getStartTime();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 0f13397..bd51c65 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -21,25 +21,21 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
 public class WindowReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 	private static final long serialVersionUID = 1L;
 	private long startTime;
 	private long nextRecordTime;
-	private Timestamp<OUT> timestamp;
+	private TimeStamp<OUT> timestamp;
 	private String nullElement = "nullElement";
 
 	public WindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
-			long slideInterval, Timestamp<OUT> timestamp) {
+			long slideInterval, TimeStamp<OUT> timestamp) {
 		super(reduceFunction, windowSize, slideInterval);
 		this.timestamp = timestamp;
-	}
-
-	@Override
-	protected void initializeAtFirstRecord() {
-		startTime = nextRecordTime - (nextRecordTime % granularity);
+		this.startTime = timestamp.getStartTime();
 	}
 
 	protected StreamRecord<OUT> getNextRecord() throws IOException {
@@ -59,7 +55,7 @@ public class WindowReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 			return false;
 		}
 	}
-	
+
 	@Override
 	protected void collectOneUnit() throws Exception {
 		OUT reduced = null;
@@ -71,9 +67,9 @@ public class WindowReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 				resetReuse();
 			}
 		}
-		if(reduced!=null){
+		if (reduced != null) {
 			state.pushBack(reduced);
-		}else{
+		} else {
 			state.pushBack(nullElement);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimeStamp.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimeStamp.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimeStamp.java
new file mode 100644
index 0000000..b6186e1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimeStamp.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util;
+
+/**
+ * Default timestamp function that uses the Java System.currentTimeMillis()
+ * method to retrieve a timestamp.
+ *
+ * @param <T>
+ *            Type of the inputs of the reducing function.
+ */
+public class DefaultTimeStamp<T> implements TimeStamp<T> {
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public long getTimestamp(T value) {
+		return System.currentTimeMillis();
+	}
+
+	@Override
+	public long getStartTime() {
+		return System.currentTimeMillis();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimestamp.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimestamp.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimestamp.java
deleted file mode 100644
index 8276a01..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/DefaultTimestamp.java
+++ /dev/null
@@ -1,34 +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.util;
-
-/**
- * Default timestamp function that uses the Java System.currentTimeMillis()
- * method to retrieve a timestamp.
- *
- * @param <T>
- *            Type of the inputs of the reducing function.
- */
-public class DefaultTimestamp<T> implements Timestamp<T> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public long getTimestamp(T value) {
-		return System.currentTimeMillis();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/TimeStamp.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/TimeStamp.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/TimeStamp.java
new file mode 100644
index 0000000..27447d7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/TimeStamp.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.util;
+
+import java.io.Serializable;
+
+/**
+ * Interface for getting a timestamp from a custom value. Used in window
+ * reduces. In order to work properly, the timestamps must be non-decreasing.
+ *
+ * @param <T>
+ *            Type of the value to create the timestamp from.
+ */
+public interface TimeStamp<T> extends Serializable {
+
+	/**
+	 * Values
+	 * 
+	 * @param value
+	 *            The value to create the timestamp from
+	 * @return The timestamp
+	 */
+	public long getTimestamp(T value);
+
+	/**
+	 * Function to define the starting time for reference
+	 * 
+	 * @return The starting timestamp
+	 */
+	public long getStartTime();
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/Timestamp.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/Timestamp.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/Timestamp.java
deleted file mode 100644
index 91758e8..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/util/Timestamp.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.api.invokable.util;
-
-import java.io.Serializable;
-
-/**
- * Interface for getting a timestamp from a custom value. Used in window
- * reduces. In order to work properly, the timestamps must be non-decreasing.
- *
- * @param <T>
- *            Type of the value to create the timestamp from.
- */
-public interface Timestamp<T> extends Serializable {
-
-	/**
-	 * Values
-	 * @param value
-	 * The value to create the timestamp from
-	 * @return The timestamp
-	 */
-	public long getTimestamp(T value);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
index 7437bec..097e391 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
@@ -25,7 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.util.MockInvokable;
 import org.apache.flink.util.Collector;
 import org.junit.Before;
@@ -45,13 +45,15 @@ public class WindowGroupReduceInvokableTest {
 		}
 	}
 
-	public static final class MyTimestamp implements Timestamp<Integer> {
+	public static final class MyTimestamp implements TimeStamp<Integer> {
 		private static final long serialVersionUID = 1L;
 
 		private Iterator<Long> timestamps;
+		private long start;
 
 		public MyTimestamp(List<Long> timestamps) {
 			this.timestamps = timestamps.iterator();
+			this.start = timestamps.get(0);
 		}
 
 		@Override
@@ -59,6 +61,11 @@ public class WindowGroupReduceInvokableTest {
 			long ts = timestamps.next();
 			return ts;
 		}
+
+		@Override
+		public long getStartTime() {
+			return start;
+		}
 	}
 
 	private final static String EOW = "|";
@@ -81,16 +88,16 @@ public class WindowGroupReduceInvokableTest {
 		slideSize = 5;
 		timestamps = Arrays.asList(101L, 103L, 121L, 122L, 123L, 124L, 180L, 181L, 185L, 190L);
 		expectedResults.add(Arrays.asList("1", "2", EOW, EOW, EOW, "3", "4", "5", "6", EOW, "3",
-				"4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", "8",
-				EOW, "7", "8", "9", EOW, "9", "10", EOW));
+				"4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", EOW, "7",
+				"8", "9", EOW, "8", "9", "10", EOW));
 		invokables.add(new WindowGroupReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
 				windowSize, slideSize, new MyTimestamp(timestamps)));
 
 		windowSize = 10;
 		slideSize = 4;
 		timestamps = Arrays.asList(101L, 103L, 110L, 112L, 113L, 114L, 120L, 121L, 125L, 130L);
-		expectedResults.add(Arrays.asList("1", "2", EOW, "3", "4", "5", EOW, "3", "4", "5", "6",
-				EOW, "4", "5", "6", "7", "8", EOW, "7", "8", "9", EOW, "7", "8", "9", EOW, "9",
+		expectedResults.add(Arrays.asList("1", "2","3" ,EOW, "3", "4", "5","6", EOW, "3", "4", "5", "6",
+				EOW, "5", "6", "7", "8", EOW, "7", "8", "9", EOW, "8","9",
 				"10", EOW));
 		invokables.add(new WindowGroupReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
 				windowSize, slideSize, new MyTimestamp(timestamps)));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
index 1aed25f..ff0951d 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
@@ -25,7 +25,7 @@ import java.util.List;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.util.MockInvokable;
 import org.junit.Test;
 
@@ -52,13 +52,18 @@ public class WindowReduceInvokableTest {
 					public Integer reduce(Integer value1, Integer value2) throws Exception {
 						return value1 + value2;
 					}
-				}, 4, 2, new Timestamp<Integer>() {
+				}, 4, 2, new TimeStamp<Integer>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
 					public long getTimestamp(Integer value) {
 						return value;
 					}
+
+					@Override
+					public long getStartTime() {
+						return 0;
+					}
 				});
 
 		List<Integer> expected = new ArrayList<Integer>();
@@ -86,14 +91,19 @@ public class WindowReduceInvokableTest {
 							Tuple2<String, Integer> value2) throws Exception {
 						return new Tuple2<String, Integer>(value1.f0, value1.f1 + value2.f1);
 					}
-				}, 3, 2, new Timestamp<Tuple2<String, Integer>>() {
+				}, 3, 2, 0, new TimeStamp<Tuple2<String, Integer>>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
 					public long getTimestamp(Tuple2<String, Integer> value) {
 						return value.f1;
 					}
-				}, 0);
+
+					@Override
+					public long getStartTime() {
+						return 1;
+					}
+				});
 
 		List<Tuple2<String, Integer>> expected2 = new ArrayList<Tuple2<String, Integer>>();
 		expected2.add(new Tuple2<String, Integer>("a", 6));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5f601cf9/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index d80b937..a433fd0 100755
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -119,12 +119,12 @@ public class IncrementalLearningSkeleton {
 
 	public static void main(String[] args) {
 
-		StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.createLocalEnvironment(PARALLELISM).setBufferTimeout(1000);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(
+				PARALLELISM).setBufferTimeout(1000);
 
 		// Build new model on every second of new data
 		DataStream<Double[]> model = env.addSource(new TrainingDataSource(), SOURCE_PARALLELISM)
-				.windowReduce(new PartialModelBuilder(), 5000);
+				.window(5000).reduceGroup(new PartialModelBuilder());
 
 		// Use partial model for prediction
 		DataStream<Integer> prediction = env.addSource(new NewDataSource(), SOURCE_PARALLELISM)


[04/18] git commit: [streaming] Reduce operator added to ConnectedDataStream + test

Posted by mb...@apache.org.
[streaming] Reduce operator added to ConnectedDataStream + test


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

Branch: refs/heads/master
Commit: 47dca692e2db320cfde14f4c6905b31972442224
Parents: d97efde
Author: gyfora <gy...@gmail.com>
Authored: Tue Sep 2 13:58:40 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../api/datastream/ConnectedDataStream.java     | 55 +++++++++++----
 .../datastream/GroupedConnectedDataStream.java  | 17 ++---
 .../operator/co/CoGroupReduceInvokable.java     |  2 +-
 .../operator/co/CoReduceInvokable.java          | 70 +++++++++++++++++++
 .../operator/co/CoStreamReduceInvokable.java    | 70 -------------------
 .../api/invokable/operator/CoReduceTest.java    | 71 ++++++++++++++++++++
 6 files changed, 193 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47dca692/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 920278c..4f17ceb 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,10 +27,12 @@ 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.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.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.CoReduceInvokable;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
 
@@ -102,12 +104,13 @@ public class ConnectedDataStream<IN1, IN2> {
 	}
 
 	/**
-	 * Applies a CoMap transformation on two separate {@link DataStream}s. The
-	 * transformation calls a {@link CoMapFunction#map1} for each element of the
-	 * first input and {@link CoMapFunction#map2} for each element of the second
-	 * input. Each CoMapFunction call returns exactly one element. The user can
-	 * also extend {@link RichCoMapFunction} to gain access to other features
-	 * provided by the {@link RichFuntion} interface.
+	 * Applies a CoMap transformation on a {@link ConnectedDataStream} and maps
+	 * the output to a common type. The transformation calls a
+	 * {@link CoMapFunction#map1} for each element of the first input and
+	 * {@link CoMapFunction#map2} for each element of the second input. Each
+	 * CoMapFunction call returns exactly one element. The user can also extend
+	 * {@link RichCoMapFunction} to gain access to other features provided by
+	 * the {@link RichFuntion} interface.
 	 * 
 	 * @param coMapper
 	 *            The CoMapFunction used to jointly transform the two input
@@ -127,13 +130,13 @@ public class ConnectedDataStream<IN1, IN2> {
 	}
 
 	/**
-	 * Applies a CoFlatMap transformation on two separate {@link DataStream}s.
-	 * The transformation calls a {@link CoFlatMapFunction#map1} for each
-	 * element of the first input and {@link CoFlatMapFunction#map2} for each
-	 * element of the second input. Each CoFlatMapFunction call returns any
-	 * number of elements including none. The user can also extend
-	 * {@link RichFlatMapFunction} to gain access to other features provided by
-	 * the {@link RichFuntion} interface.
+	 * Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
+	 * maps the output to a common type. The transformation calls a
+	 * {@link CoFlatMapFunction#map1} for each element of the first input and
+	 * {@link CoFlatMapFunction#map2} for each element of the second input. Each
+	 * CoFlatMapFunction call returns any number of elements including none. The
+	 * user can also extend {@link RichFlatMapFunction} to gain access to other
+	 * features provided by the {@link RichFuntion} interface.
 	 * 
 	 * @param coFlatMapper
 	 *            The CoFlatMapFunction used to jointly transform the two input
@@ -153,6 +156,32 @@ public class ConnectedDataStream<IN1, IN2> {
 				outTypeWrapper, new CoFlatMapInvokable<IN1, IN2, OUT>(coFlatMapper));
 	}
 
+	/**
+	 * 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.
+	 * 
+	 * @param coReducer
+	 *            The {@link CoReduceFunction} that will be called for every
+	 *            element of the inputs.
+	 * @return The transformed DataStream.
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> reduce(CoReduceFunction<IN1, IN2, OUT> coReducer) {
+
+		FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coReducer,
+				CoReduceFunction.class, 0);
+		FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coReducer,
+				CoReduceFunction.class, 1);
+		FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coReducer,
+				CoReduceFunction.class, 2);
+
+		return addCoFunction("coReduce", coReducer, in1TypeWrapper, in2TypeWrapper, outTypeWrapper,
+				new CoReduceInvokable<IN1, IN2, OUT>(coReducer));
+	}
+
 	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/47dca692/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 13ba3ab..8276561 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
@@ -37,16 +37,17 @@ public class GroupedConnectedDataStream<IN1, IN2> extends ConnectedDataStream<IN
 	}
 
 	/**
-	 * Applies a CoReduce transformation on the grouped data stream grouped on
-	 * by the given key position. 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 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.
 	 * 
 	 * @param coReducer
-	 *            The {@link CoReduceFunction} that will be called for every two
+	 *            The {@link CoReduceFunction} that will be called for every
 	 *            element with the same key of each input DataStream.
 	 * @return The transformed DataStream.
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47dca692/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
index d7ad32e..ab827d8 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoGroupReduceInvokable.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.invokable.operator.co;
 import org.apache.flink.streaming.api.function.co.CoReduceFunction;
 import org.apache.flink.streaming.state.MutableTableState;
 
-public class CoGroupReduceInvokable<IN1, IN2, OUT> extends CoStreamReduceInvokable<IN1, IN2, OUT> {
+public class CoGroupReduceInvokable<IN1, IN2, OUT> extends CoReduceInvokable<IN1, IN2, OUT> {
 	private static final long serialVersionUID = 1L;
 
 	private int keyPosition1;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47dca692/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
new file mode 100644
index 0000000..407f217
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoReduceInvokable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator.co;
+
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+
+public class CoReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
+	private static final long serialVersionUID = 1L;
+
+	protected CoReduceFunction<IN1, IN2, OUT> coReducer;
+	protected IN1 currentValue1 = null;
+	protected IN2 currentValue2 = null;
+	protected IN1 nextValue1 = null;
+	protected IN2 nextValue2 = null;
+
+	public CoReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer) {
+		super(coReducer);
+		this.coReducer = coReducer;
+		currentValue1 = null;
+		currentValue2 = null;
+	}
+
+	@Override
+	public void handleStream1() throws Exception {
+		nextValue1 = reuse1.getObject();
+		callUserFunctionAndLogException1();
+	}
+
+	@Override
+	public void handleStream2() throws Exception {
+		nextValue2 = reuse2.getObject();
+		callUserFunctionAndLogException2();
+	}
+
+	@Override
+	protected void callUserFunction1() throws Exception {
+		if (currentValue1 != null) {
+			currentValue1 = coReducer.reduce1(currentValue1, nextValue1);
+		} else {
+			currentValue1 = nextValue1;
+		}
+		collector.collect(coReducer.map1(currentValue1));
+	}
+
+	@Override
+	protected void callUserFunction2() throws Exception {
+		if (currentValue2 != null) {
+			currentValue2 = coReducer.reduce2(currentValue2, nextValue2);
+		} else {
+			currentValue2 = nextValue2;
+		}
+		collector.collect(coReducer.map2(currentValue2));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47dca692/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
deleted file mode 100644
index e4acb99..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoStreamReduceInvokable.java
+++ /dev/null
@@ -1,70 +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.operator.co;
-
-import org.apache.flink.streaming.api.function.co.CoReduceFunction;
-
-public abstract class CoStreamReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
-	private static final long serialVersionUID = 1L;
-
-	protected CoReduceFunction<IN1, IN2, OUT> coReducer;
-	protected IN1 currentValue1 = null;
-	protected IN2 currentValue2 = null;
-	protected IN1 nextValue1 = null;
-	protected IN2 nextValue2 = null;
-
-	public CoStreamReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer) {
-		super(coReducer);
-		this.coReducer = coReducer;
-		currentValue1 = null;
-		currentValue2 = null;
-	}
-
-	@Override
-	public void handleStream1() throws Exception {
-		nextValue1 = reuse1.getObject();
-		callUserFunctionAndLogException1();
-	}
-
-	@Override
-	public void handleStream2() throws Exception {
-		nextValue2 = reuse2.getObject();
-		callUserFunctionAndLogException2();
-	}
-
-	@Override
-	protected void callUserFunction1() throws Exception {
-		if (currentValue1 != null) {
-			currentValue1 = coReducer.reduce1(currentValue1, nextValue1);
-		} else {
-			currentValue1 = nextValue1;
-		}
-		collector.collect(coReducer.map1(currentValue1));
-	}
-
-	@Override
-	protected void callUserFunction2() throws Exception {
-		if (currentValue2 != null) {
-			currentValue2 = coReducer.reduce2(currentValue2, nextValue2);
-		} else {
-			currentValue2 = nextValue2;
-		}
-		collector.collect(coReducer.map2(nextValue2));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47dca692/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java
new file mode 100755
index 0000000..4f52444
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoReduceTest.java
@@ -0,0 +1,71 @@
+/** Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoReduceInvokable;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.junit.Test;
+
+public class CoReduceTest {
+
+	public static class MyCoReduceFunction implements CoReduceFunction<Integer, String, Integer> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Integer reduce1(Integer value1, Integer value2) {
+			return value1 * value2;
+		}
+
+		@Override
+		public String reduce2(String value1, String value2) {
+			return value1 + value2;
+		}
+
+		@Override
+		public Integer map1(Integer value) {
+			return value;
+		}
+
+		@Override
+		public Integer map2(String value) {
+			return Integer.parseInt(value);
+		}
+
+	}
+
+	@Test
+	public void coGroupReduceTest() {
+
+		CoReduceInvokable<Integer, String, Integer> coReduce = new CoReduceInvokable<Integer, String, Integer>(
+				new MyCoReduceFunction());
+
+		List<Integer> expected1 = Arrays.asList(1, 9, 2, 99, 6, 998, 24);
+
+		assertEquals(
+				expected1,
+				(MockCoInvokable.createAndExecute(coReduce, Arrays.asList(1, 2, 3, 4),
+						Arrays.asList("9", "9", "8"))));
+
+	}
+}


[07/18] git commit: [FLINK-1058] [streaming] Streaming initial documentation

Posted by mb...@apache.org.
[FLINK-1058] [streaming] Streaming initial documentation


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

Branch: refs/heads/master
Commit: 13a9277209cb5248e6e7f7ef9aa895f4103c591c
Parents: bcbebed
Author: mbalassi <ba...@gmail.com>
Authored: Tue Sep 2 18:06:32 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md | 978 +++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 978 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/13a92772/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
new file mode 100644
index 0000000..04e2f2e
--- /dev/null
+++ b/docs/streaming_guide.md
@@ -0,0 +1,978 @@
+---
+title: "Flink Stream Processing API"
+---
+
+<section id="top">
+Flink Streaming
+=======
+
+Flink Streaming is an extension of the core Flink API for high-throughput, low-latency data stream processing. The system can connect to and process data streams from many data sources like Flume, Twitter, ZeroMQ and also from any user defined data source. Data streams can be transformed and modified using high-level functions similar to the ones provided by the batch processing API. Flink Streaming provides native support for iterative stream processing. The processed data can be pushed to different output types.
+
+Flink Streaming API
+-----------
+
+The Streaming API is part of the *addons* Maven project. All relevant classes are located in the *org.apache.flink.streaming* package.
+
+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>
+</dependency>
+```
+
+Create a data stream flow with our Java API as described below. In order to create your own Flink Streaming program, we encourage you to start with the [skeleton](#skeleton) and gradually add your own [operations](#operations). The remaining sections act as references for additional operations and advanced features.
+
+<section id="toc">
+<div id="docs_05_toc">
+  <div class="list-group">
+{% for sublink in page.toc %}
+   <a href="#{{ sublink.anchor }}" class="list-group-item">{{forloop.index}}. <strong>{{ sublink.title }}</strong></a>
+{% endfor %}
+  </div>
+</div>
+
+<section id="example">
+Example Program
+---------------
+
+The following program is a complete, working example of streaming WordCount. You can copy &amp; paste the code to run it locally.
+
+```java
+public class StreamingWordCount {
+
+	
+	public static void main(String[] args) {
+
+		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));
+			}
+		}
+	}
+	
+}
+```
+
+[Back to top](#top)
+
+<section id="skeleton">
+Program Skeleton
+----------------
+
+As we could already see in the example, a Flink Streaming program looks almost identical to a regular Flink program. Each stream processing program consists of the following parts:
+
+1. Creating a `StreamExecutionEnvironment`,
+2. Connecting to data stream sources,
+3. Specifying transformations on the data streams,
+4. Specifying output for the processed data,
+5. Executing the program.
+
+As these steps are basically the same as in the core API we will only note the important differences.
+For stream processing jobs, the user needs to obtain a `StreamExecutionEnvironment` in contrast with the batch API where one would need an `ExecutionEnvironment`. The process otherwise is essentially the same:
+
+```java 
+StreamExecutionEnvironment.createLocalEnvironment(params…)
+StreamExecutionEnvironment.createRemoteEnvironment(params…)
+```
+
+For connecting to data streams the `StreamExecutionEnvironment` has many different methods, from basic file sources to completely general user defined data sources. We will go into details in the [basics](#basics) section.
+
+```java
+env.readTextFile(filePath)
+```
+
+After defining the data stream sources, the user can specify transformations on the data streams to create a new data stream. Different data streams can be also combined together for joint transformations which we will see in the [operations](#operations) section.
+
+```java
+dataStream.map(new Mapper()).reduce(new Reducer())
+```
+
+The processed data can be pushed to different outputs called sinks. The user can define their own sinks or use any predefined filesystem or database sink.
+
+```java
+dataStream.writeAsCsv(path)
+```
+
+Once the complete program is specified `execute()` needs to be called on the `StreamExecutionEnvironment`. This will either execute on the local machine or submit the program for execution on a cluster, depending on the chosen execution environment.
+
+```java
+env.execute()
+```
+
+[Back to top](#top)
+
+<section id="basics">
+Basics
+----------------
+
+### DataStream
+
+The `DataStream` is the basic abstraction provided by the the Flink Streaming API. It represents a continuous stream of data of a certain type from either a data source or a transformed data stream. Operations on the DataStreams will be applied on individual data points or windows of the `DataStream` based on the type of the operation. For example the map operator transforms each data point individually while window or batch aggregations work on an interval of data points at the same time.
+ 
+The different operations return different `DataStream` types allowing more elaborate transformations, for example the `groupBy()` method returns a `GroupedDataStream` which can be used for group operations.
+
+### Partitioning
+
+Partitioning controls how individual data points are distributed among the parallel instances of the transformation operators. By default Forward partitioning is used. There are several partitioning types supported in Flink Streaming:
+
+ * Forward: Forward partitioning directs the output data to the next operator on the same core (if possible) avoiding expensive network I/O. This is the default partitioner.
+Usage: `dataStream.forward()`
+ * Shuffle: Shuffle partitioning randomly partitions the output data stream to the next operator using uniform distribution.
+Usage: `dataStream.shuffle()`
+ * Distribute: Distribute partitioning directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution.
+Usage: `dataStream.distribute()`
+ * Field: Field partitioning partitions the output data stream based on the hash code of a selected key field. Data points with the same key will always go to the same operator instance.
+Usage: `dataStream.partitionBy(keyposition)`
+ * Broadcast: Broadcast partitioning sends the output data stream to all parallel instances of the next operator.
+Usage: `dataStream.broadcast()`
+ * Global: All data points end up at the same operator instance. To achieve a clearer structure use the parallelism setting of the corresponding operator for this.
+Usage: `operator.setParallelism(1)`
+
+### Sources
+
+The user can connect to different data streams by the different implemenations of `DataStreamSource` using methods provided in `StreamExecutionEnvironment`. There are several predefined ones similar to the ones provided by the batch API like:
+
+ * `env.genereateSequence(from, to)`
+ * `env.fromElements(elements…)`
+ * `env.fromCollection(collection)`
+ * `env.readTextFile(filepath)`
+
+These can be used to easily test and debug streaming programs. There are also some streaming specific sources for example `env.readTextStream(filepath)` which iterates over the same file infinitely providing yet another nice testing tool.
+There are implemented connectors for a number of the most popular message queue services, please refer to the section on [connectors](#connectors) for more detail.
+Besides the pre-defined solutions the user can implement their own source by implementing the `SourceFunction` interface and using the `env.addSource(sourceFunction)` method of the `StreamExecutionEnvironment`.
+
+### 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:
+ * `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
+
+The user can also implement arbitrary sink functionality by implementing the `SinkFunction` interface and using it with `dataStream.addSink(sinkFunction)`.
+
+[Back to top](#top)
+
+<section id="operations">
+Operations
+----------------
+
+Operations represent transformations on the `DataStream`. The user can chain and combine multiple operators on the data stream to produce the desired processing steps. Most of the operators work very similar to the core Flink API allowing developers to reason about `DataStream` the same way as they would about `DataSet`. At the same time there are operators that exploit the streaming nature of the data to allow advanced functionality.
+
+### Basic operators
+
+Basic operators can be seen as functions that transform each data element in the data stream.
+ 
+#### Map
+The Map transformation applies a user-defined `MapFunction` on each element of a `DataStream`. It implements a one-to-one mapping, that is, exactly one element must be returned by the function.
+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;
+			}
+		})
+```
+
+#### FlatMap
+The FlatMap transformation applies a user-defined `FlatMapFunction` on each element of a `DataStream`. This variant of a map function can return arbitrary many result elements (including none) for each input element.
+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);
+				}
+			}
+		})
+```
+
+#### Filter
+The Filter transformation applies a user-defined `FilterFunction` on each element of a `DataStream` and retains only those elements for which the function returns true.
+A filter that filters out zero values:
+
+```java
+dataStream.filter(new FilterFunction<Integer>() {		
+			@Override
+			public boolean filter(Integer value) throws Exception {
+				return value != 0;
+			}
+		})
+```
+
+#### Reduce
+The Reduce transformation applies a user-defined `ReduceFunction` to all elements of a `DataStream`. The `ReduceFunction` subsequently combines pairs of elements into one element and outputs the current reduced value as a `DataStream`.
+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;
+			}
+		})
+```
+
+#### Merge
+Merges two or more `DataStream` instances creating a new DataStream containing all the elements from all the streams.
+
+```java
+dataStream.merge(otherStream1, otherStream2…)
+```
+
+### Window/Batch operators
+
+Window and batch operators allow the user to execute function on slices or windows of the DataStream in a sliding fashion. If the stepsize for the slide is not defined then the window/batchsize is used as stepsize by default.
+
+#### Window reduce
+The transformation calls a user-defined `GroupReduceFunction` on records received during the predefined time window. The window is shifted after each reduce call.
+A window reduce that sums the elements in the last minute with 10 seconds stepsize:
+
+```java
+dataStream.windowReduce(new GroupReduceFunction<Integer, Integer>() {
+			@Override
+			public void reduce(Iterable<Integer> values, Collector<Integer> out) throws Exception {
+				Integer sum = 0;
+				for(Integer val: values){
+					sum+=val;
+				}
+			}
+		}, 60000, 10000);
+```
+
+#### Batch reduce
+The transformation calls a `GroupReduceFunction` for each data batch of the predefined size. The batch slides by the predefined number of elements after each call. Works similarly to window reduce.
+
+```java
+dataStream.batchReduce(reducer, batchSize, slideSize)
+```
+
+### Grouped operators
+
+Some transformations require that the `DataStream` is grouped on some key value. The user can create a `GroupedDataStream` by calling the `groupBy(keyPosition)` method of a non-grouped `DataStream`. The user can apply different reduce transformations on the obtained `GroupedDataStream`:
+
+#### Reduce on GroupedDataStream
+When the reduce operator is applied on a grouped data stream, the user-defined `ReduceFunction` will combine subsequent pairs of elements having the same key value. The combined results are sent to the output stream.
+
+#### Window/Batchreduce on GroupedDataStream
+Similarly to the grouped reduce operator the window and batch reduce operators work the same way as in the non-grouped case except that in a data window/batch every `GroupReduceFunction` call will receive data elements for only the same keys.
+
+### Co operators
+
+Co operators allow the users to jointly transform two `DataStreams` of different types providing a simple way to jointly manipulate a shared state. It is designed to support joint stream transformations where merging is not appropriate due to different data types or the in cases when user needs explicit track of the datas origin.
+Co operators can be applied to `ConnectedDataStreams` which represent two `DataStreams` of possibly different types. A `ConnectedDataStream` can be created by calling the `connect(otherDataStream)` method of a `DataStream`. Please note that the two connected `DataStreams` can also be merged data streams.
+
+#### Map on ConnectedDataStream
+Applies a CoMap transformation on two separate DataStreams, mapping them to a common output type. The transformation calls a `CoMapFunction.map1()` for each element of the first input and `CoMapFunction.map2()` for each element of the second input. Each CoMapFunction call returns exactly one element.
+A CoMap operator that outputs true if an Integer value is received and false if a String value is received:
+
+```java
+DataStream<Integer> ds1 = ...
+DataStream<String> ds2 = ...
+		
+ds1.connect(ds2).
+	.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
+The FlatMap operator for the `ConnectedDataStream` works similarly to CoMap, but instead of returning exactly one element after each map call the user can output zero or more values using the Collector interface. 
+
+```java
+DataStream<Integer> ds1 = ...
+DataStream<String> ds2 = ...
+
+ds1.connect(ds2)
+	.flatMap(new CoFlatMapFunction<Integer, String, Boolean>() {
+
+			@Override
+			public void flatMap1(Integer value, Collector<Boolean> out) {
+				out.collect(true);
+			}
+
+			@Override
+			public void flatMap2(String value, Collector<Boolean> out) {
+				out.collect(false);
+			}
+		})
+```
+
+#### Reduce on ConnectedDataStream
+The Reduce operator for the `ConnectedDataStream` applies a simple reduce transformation on the joined data streams and then maps the reduced elements to a common output type.
+
+<section id="output-splitting">
+### Output splitting
+
+Most data stream operators support directed outputs. It means that different data elements are received by only given outputs. The outputs are referenced by their name given at the point of receiving:
+
+```java
+SplitDataStream<Integer> split = someDataStream.split(outputSelector);
+DataStream<Integer> even = split.select("even");
+DataStream<Integer> odd = split.select("odd");
+```
+
+Data streams will only receive the elements directed to selected output names. These outputs are directed by implementing a selector function (extending `OutputSelector`):
+
+```java
+void select(OUT value, Collection<String> outputs);
+```
+
+The data is sent to all the outputs added to the collection outputs (referenced by their name). This way the direction of the outputs can be determined by the value of the data sent. For example:
+
+```java
+@Override
+void select(Integer value, Collection<String> outputs) {
+    if (value % 2 == 0) {
+        outputs.add("even");
+    } else {
+        outputs.add("odd");
+    }
+}
+```
+
+This output selection allows data streams to listen to multiple outputs, and data points to be sent to multiple outputs. A value is sent to all the outputs specified in the `OutputSelector` and a data stream will receive a value if it has selected any of the outputs the value is sent to. The stream will receive the data at most once.
+It is common that a stream needs to listen to all the outputs, so `split.selectAll()` is provided as an alias for explicitly selecting all output names.
+
+
+### Iterations
+The Flink Streaming API supports implementing iterative stream processing dataflows similarly to the core Flink API. Iterative streaming programs also implement a step function and embed it into an `IterativeDataStream`.
+Unlike in the core API the user does not define the maximum number of iterations, but at the tail of each iteration the output is both streamed forward to the next operator and also streamed back to the iteration head. The user controls the output of the iteration tail using [output splitting](#output-splitting).
+To start an iterative part of the program the user defines the iteration starting point:
+
+```java
+IterativeDataStream<Integer> iteration = source.iterate();
+```
+The operator applied on the iteration starting point will be the head of the iteration, where data is fed back from the iteration tail.
+
+```java
+DataStream<Integer> head = iteration.map(new IterationHead());
+```
+
+To close an iteration and define the iteration tail, the user needs to call `.closeWith(tail)` method of the `IterativeDataStream`:
+
+```java
+DataStream<Integer> tail = head.map(new IterationTail());
+iteration.closeWith(tail);
+```
+Or to use with output splitting:
+```java
+SplitDataStream<Integer> tail = head.map(new IterationTail()).split(outputSelector);
+iteration.closeWith(tail.select("iterate"));
+``` 
+
+Because iterative streaming programs do not have a set number of iteratons for each data element, the streaming program no information on the end of its input. From this it follows that iterative streaming programs run until the user manually stops the program. While this is acceptable under normal circumstances we provide a method to allow iterative programs to shut down automatically if no input received by the iteration head for a predefined number of milliseconds.
+To use this function the user need to call, the `iteration.setMaxWaitTime(millis)` to control the max wait time. 
+
+### Rich functions
+The usage of rich functions are essentially the same as in the core Flink API. All transformations that take as argument a user-defined function can instead take as argument a rich function:
+
+```java
+ds.map(new RichMapFunction<String, Integer>() {
+  public Integer map(String value) { return value.toString(); }
+});
+```
+
+Rich functions provide, in addition to the user-defined function (`map`, `reduce`, etc), the `open` and `close` methods for initialization and finalization. (In contrast to the core API, the streaming API currently does not support the  `getRuntimeContext` and `setRuntimeContext` methods.)
+
+[Back to top](#top)
+
+<section id="Operator-settings">
+Operator Settings
+----------------
+
+### Parallelism
+
+Setting parallelism for operators works exactly the same way as in the core Flink API The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(dop)` method.
+
+### Buffer timeout
+
+By default data points are not transferred on the network one-by-one, which would cause unnecessary network traffic, but are buffered in the output buffers. The size of the output buffers can be set in the Flink config files. While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough.
+To tackle this issue the user can call `env.setBufferTimeout(timeoutMillis)` on the execution environment (or on individual operators) to set a maximum wait time for the buffers to fill up. After this time the buffers are flushed automatically even if they are not full. Usage:
+
+```java
+LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+env.setBufferTimeout(timeoutMillis);
+
+env.genereateSequence(1,10).map(new MyMapper()).setBufferTimeout(timeoutMillis);
+```
+
+### Mutability
+
+Most operators allows setting mutability for reading input data. If the operator is set mutable then the variable used to store input data for operators will be reused in a mutable fashion to avoid excessive object creation. By default, all operators are set to immutable.
+Usage:
+```java
+operator.setMutability(isMutable)
+```
+
+[Back to top](#top)
+	
+<section id="connectors">
+Stream connectors
+----------------
+
+Connectors provide an interface for accessing data from various third party sources (message queues). Currently four connectors are natively supported, namely [Apache Kafka](https://kafka.apache.org/),  [RabbitMQ](http://www.rabbitmq.com/), [Apache Flume](https://flume.apache.org/index.html) and [Twitter Streaming API](https://dev.twitter.com/docs/streaming-apis).
+
+Typically the connector packages consist of an abstract source and sink (with the exception of Twitter where only a source is provided). The burden of the user is to implement a subclass of these abstract classes specifying a serializer and a deserializer function. 
+
+To run an application using one of these connectors usually additional third party components are required to be installed and launched, e.g. the servers for the message queues. Further instructions for these can be found in the corresponding subsections. [Docker containers](#docker_connectors) are also provided encapsulating these services to aid users getting started with connectors.
+
+### Apache Kafka
+
+This connector provides access to data streams from [Apache Kafka](https://kafka.apache.org/).
+
+#### Installing Apache Kafka
+* Follow the instructions from [Kafka's quickstart](https://kafka.apache.org/documentation.html#quickstart) to download the code and launch a server (launching a Zookeeper and a Kafka server is required every time before starting the application).
+* On 32 bit computers [this](http://stackoverflow.com/questions/22325364/unrecognized-vm-option-usecompressedoops-when-running-kafka-from-my-ubuntu-in) problem may occur. 
+* If the Kafka zookeeper and server are running on a remote machine then in the config/server.properties file the advertised.host.name must be set to the machine's IP address.
+
+#### 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.
+
+The implemented class must extend `KafkaSource`, for example: `KafkaSource<Tuple1<String>>`.
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+public MyKafkaSource(String zkQuorum, String groupId, String topicId, int numThreads) {
+	super(zkQuorum, groupId, topicId, numThreads);
+}
+```
+
+##### Deserializer
+An example of an implementation of a deserializer:
+
+```java
+@Override
+public Tuple1<String> deserialize(byte[] msg) {
+	String s = new String(msg);
+	if(s.equals("q")){
+		closeWithoutSend();
+	}
+	return new Tuple1<String>(s);
+}
+```
+
+The source closes when it receives the String `"q"`.
+
+###### Close<a name="kafka_source_close"></a>
+Two types of close functions are available, namely `closeWithoutSend()` and `sendAndClose()`. The former closes the connection immediately and no further data will be sent, while the latter closes the connection only when the next message is sent after this call.
+
+In the example provided `closeWithoutSend()` is used because here the String `"q"` is meta-message indicating the end of the stream and there is no need to forward it. 
+
+#### 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.
+
+The implemented class must extend `KafkaSink`, for example `KafkaSink<Tuple1<String>, String>`.
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+public MyKafkaSink(String topicId, String brokerAddr) {
+	super(topicId, brokerAddr);
+}
+```
+
+##### Serializer
+An example of an implementation of a serializer:
+
+```java
+@Override
+public String serialize(Tuple1<String> tuple) {
+	if(tuple.f0.equals("q")){
+		sendAndClose();
+	}
+	return tuple.f0;
+}
+```
+
+##### Close
+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.
+	addSource(new MyKafkaSource("localhost:2181", "group", "test", 1), SOURCE_PARALELISM)
+	.print();
+```
+
+The followings have to be provided for the `MyKafkaSource()` constructor in order:
+
+1. The hostname
+2. The group name
+3. The topic name
+4. The parallelism
+
+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
+	.addSource(new MySource())
+	.addSink(new MyKafkaSink("test", "localhost:9092"));
+```
+
+The followings have to be provided for the `MyKafkaSink()` constructor in order:
+
+1. The topic name
+2. The hostname
+
+More about Kafka can be found [here](https://kafka.apache.org/documentation.html).
+
+[Back to top](#top)
+
+### Apache Flume
+
+This connector provides access to datastreams from [Apache Flume](http://flume.apache.org/).
+
+#### Installing Apache Flume
+[Download](http://flume.apache.org/download.html) Apache Flume. A configuration file is required for starting agents in Flume. A configuration file for running the example can be found [here](#config_file). 
+
+#### 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.
+
+The implemented class must extend `FlumeSource` for example: `FlumeSource<Tuple1<String>>`
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+MyFlumeSource(String host, int port) {
+	super(host, port);
+}
+```
+
+##### Deserializer
+An example of an implementation of a deserializer:
+
+```java
+@Override
+public Tuple1<String> deserialize(byte[] msg) {
+	String s = (String) SerializationUtils.deserialize(msg);
+	Tuple1<String> out = new Tuple1<String>();
+	out.f0 = s;
+	if (s.equals("q")) {
+		closeWithoutSend();
+	}
+	return out;
+}
+```
+
+The source closes when it receives the String `"q"`.
+
+##### Close<a name="flume_source_close"></a>
+Two types of close functions are available, namely `closeWithoutSend()` and `sendAndClose()`.The former closes the connection immediately and no further data will be sent, while the latter closes the connection only when the next message is sent after this call.
+
+In the example `closeWithoutSend()` is used because here the String `"q"` is meta-message indicating the end of the stream and there is no need to forward it. 
+
+#### 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.
+
+The implemented class must extend `FlumeSink`, for example `FlumeSink<Tuple1<String>, String>`.
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+public MyFlumeSink(String host, int port) {
+	super(host, port);
+}
+```
+
+##### Serializer
+An example of an implementation of a serializer.
+
+```java
+@Override
+public byte[] serialize(Tuple1<String> tuple) {
+	if (tuple.f0.equals("q")) {
+		try {
+			sendAndClose();
+		} catch (Exception e) {
+			new RuntimeException("Error while closing Flume connection with " + port + " at "
+				+ host, e);
+		}
+	}
+	return SerializationUtils.serialize(tuple.f0);
+}
+```
+
+##### Close
+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
+	.addSource(new MyFlumeSource("localhost", 41414))
+	.print();
+```
+
+The followings have to be provided for the `MyFlumeSource()` constructor in order:
+
+1. The hostname
+2. The port number
+
+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
+	.fromElements("one", "two", "three", "four", "five", "q")
+	.addSink(new MyFlumeSink("localhost", 42424));
+```
+
+The followings have to be provided for the `MyFlumeSink()` constructor in order:
+
+1. The hostname
+2. The port number
+
+##### Configuration file<a name="config_file"></a>
+An example of a configuration file:
+
+```
+a1.channels = c1
+a1.sources = r1
+a1.sinks = k1
+
+a1.channels.c1.type = memory
+
+a1.sources.r1.channels = c1
+a1.sources.r1.type = avro
+a1.sources.r1.bind = localhost
+a1.sources.r1.port = 42424
+
+a1.sinks.k1.channel = c1
+a1.sinks.k1.type = avro
+a1.sinks.k1.hostname = localhost
+a1.sinks.k1.port = 41414
+```
+
+To run the `FlumeTopology` example the previous configuration file must located in the Flume directory and named example.conf and the agent can be started with the following command:
+
+```
+bin/flume-ng agent --conf conf --conf-file example.conf --name a1 -Dflume.root.logger=INFO,console
+```
+
+If the agent is not started before the application starts a `FlumeSink` then the sink will retry to build the connection for 90 seconds, if unsuccessful it throws a `RuntimeException`.
+
+More on Flume can be found [here](http://flume.apache.org).
+
+[Back to top](#top)
+
+### RabbitMQ
+
+This connector provides access to datastreams from [RabbitMQ](http://www.rabbitmq.com/).
+
+##### Installing RabbitMQ
+Follow the instructions from the [RabbitMQ download page](http://www.rabbitmq.com/download.html). After the installation the server automatically starts and the application connecting to RabbitMQ can be launched.
+
+#### RabbitMQ Source
+An abstract class providing an interface for receiving data from RabbitMQ. 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 RabbitMQ,
+* Stop the source manually when necessary with one of the close functions.
+
+The implemented class must extend `RabbitMQSource` for example: `RabbitMQSource<Tuple1<String>>`
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+public MyRMQSource(String HOST_NAME, String QUEUE_NAME) {
+	super(HOST_NAME, QUEUE_NAME);
+}
+```
+
+##### Deserializer
+An example of an implemetation of a deserializer:
+
+```java
+@Override
+public Tuple1<String> deserialize(byte[] t) {
+	String s = (String) SerializationUtils.deserialize(t);
+	Tuple1<String> out = new Tuple1<String>();
+	out.f0 = s;
+	if (s.equals("q")) {
+		closeWithoutSend();
+	}
+	return out;
+}
+```
+
+The source closes when it receives the String `"q"`.
+
+##### Close<a name="rmq_source_close"></a>
+Two types of close functions are available, namely `closeWithoutSend()` and `sendAndClose()`. The former closes the connection immediately and no further data will be sent, while the latter closes the connection only when the next message is sent after this call.
+
+Closes the connection only when the next message is sent after this call.
+
+In the example `closeWithoutSend()` is used because here the String `"q"` is meta-message indicating the end of the stream and there is no need to forward it. 
+
+#### 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
+
+The implemented class must extend `RabbitMQSink` for example: `RabbitMQSink<Tuple1<String>, String>`
+
+##### Constructor
+An example of an implementation of a constructor:
+
+```java
+public MyRMQSink(String HOST_NAME, String QUEUE_NAME) {
+	super(HOST_NAME, QUEUE_NAME);
+}
+```
+
+##### Serializer
+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);
+}
+```
+
+##### Close
+The API provided is the [same](#rmq_source_close) as the one for `RabbitMQSource`.
+
+#### Building A Topology
+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<Tuple1<String>> dataStream1 = env
+	.addSource(new MyRMQSource("localhost", "hello"))
+	.print();
+```
+
+The followings have to be provided for the `MyRabbitMQSource()` constructor in order:
+
+1. The hostname
+2. The queue name
+
+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
+	.fromElements("one", "two", "three", "four", "five", "q")
+	.addSink(new MyRMQSink("localhost", "hello"));
+```
+
+The followings have to be provided for the `MyRabbitMQSink()` constructor in order:
+
+1. The hostname
+1. The queue name
+
+More about RabbitMQ can be found [here](http://www.rabbitmq.com/).
+
+[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. 
+
+#### RabbitMQ
+Pull the image:
+
+```batch
+sudo docker pull flinkstreaming/flink-connectors-rabbitmq 
+```
+
+To run the container type:
+
+```batch
+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.
+
+To start the RabbitMQ server:
+
+```batch
+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
+```
+
+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)
+```/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:
+
+```
+<DATE> INFO rabbitmq.RMQTopology: String: <one> arrived from RMQ 
+<DATE> INFO rabbitmq.RMQTopology: String: <two> arrived from RMQ
+<DATE> INFO rabbitmq.RMQTopology: String: <three> arrived from RMQ
+<DATE> INFO rabbitmq.RMQTopology: String: <four> arrived from RMQ
+<DATE> INFO rabbitmq.RMQTopology: String: <five> arrived from RMQ
+```
+
+#### Apache Kafka
+
+Pull the image:
+
+```batch
+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
+```
+
+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.
+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 &
+```
+
+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 &
+```
+
+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
+```
+
+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)
+```/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:
+
+```
+<DATE> INFO kafka.KafkaTopology: String: (0) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (1) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (2) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (3) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (4) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (5) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (6) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (7) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (8) arrived from Kafka
+<DATE> INFO kafka.KafkaTopology: String: (9) arrived from Kafka
+```
+
+#### 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.
+
+Pull the image:
+
+```batch
+sudo docker pull flinkstreaming/flink-connectors-flume
+```
+
+To run the container type:
+
+```batch
+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.
+
+To have the latest version of Flink type:
+```batch
+cd /git/incubator-flink/
+git pull
+```
+
+Then build the code with:
+
+```batch
+cd /git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/
+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 &
+```
+
+Then press enter and launch the example with:
+
+```batch
+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)
+```/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:
+
+```
+<DATE> INFO flume.FlumeTopology: String: <one> arrived from Flume
+<DATE> INFO flume.FlumeTopology: String: <two> arrived from Flume
+<DATE> INFO flume.FlumeTopology: String: <three> arrived from Flume
+<DATE> INFO flume.FlumeTopology: String: <four> arrived from Flume
+<DATE> INFO flume.FlumeTopology: String: <five> arrived from Flume
+```
+
+[Back to top](#top)


[08/18] git commit: [FLINK-1080] [streaming] Streaming aggregation update and refactor

Posted by mb...@apache.org.
[FLINK-1080] [streaming] Streaming aggregation update and refactor


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

Branch: refs/heads/master
Commit: bcbebed01537ee7a414cd51428f4a3b8e8fc23e1
Parents: 4d73f51
Author: gyfora <gy...@gmail.com>
Authored: Fri Sep 5 16:07:59 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    | 116 +++++++++++---
 .../api/datastream/GroupedDataStream.java       |  71 +++++++--
 .../aggregation/AggregationFunction.java        |  34 +++++
 .../ComparableAggregationFunction.java          |  12 +-
 .../aggregation/MaxAggregationFunction.java     |  32 ++++
 .../aggregation/MinAggregationFunction.java     |  32 ++++
 .../StreamingAggregationFunction.java           |  45 ------
 .../StreamingMaxAggregationFunction.java        |  32 ----
 .../StreamingMinAggregationFunction.java        |  32 ----
 .../StreamingSumAggregationFunction.java        |  64 --------
 .../aggregation/SumAggregationFunction.java     | 150 +++++++++++++++++++
 .../operator/BatchGroupReduceInvokable.java     |   1 -
 .../api/invokable/operator/BatchIterator.java   |   3 +-
 .../operator/WindowGroupReduceInvokable.java    |   1 -
 .../streaming/api/AggregationFunctionTest.java  |  92 +++++++++---
 .../api/invokable/operator/CoFlatMapTest.java   |   6 -
 16 files changed, 479 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/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 28d07d6..d78ceae 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
@@ -34,12 +34,13 @@ import org.apache.flink.api.java.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichMapFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
+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.function.sink.PrintSinkFunction;
 import org.apache.flink.streaming.api.function.sink.SinkFunction;
 import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
@@ -171,7 +172,43 @@ public abstract class DataStream<OUT> {
 	public TypeInformation<OUT> getOutputType() {
 		return this.outTypeWrapper.getTypeInfo();
 	}
-	
+
+	/**
+	 * Gets the class of the field at the given position
+	 * 
+	 * @param pos
+	 *            Position of the field
+	 * @return The class of the field
+	 */
+	@SuppressWarnings("rawtypes")
+	protected Class<?> getClassAtPos(int pos) {
+		Class<?> type;
+		TypeInformation<OUT> outTypeInfo = outTypeWrapper.getTypeInfo();
+		if (outTypeInfo.isTupleType()) {
+			type = ((TupleTypeInfo) outTypeInfo).getTypeAt(pos).getTypeClass();
+		} else if (pos == 0) {
+			type = outTypeInfo.getTypeClass();
+		} else {
+			throw new IndexOutOfBoundsException("Position is out of range");
+		}
+		return type;
+	}
+
+	/**
+	 * Checks if the given field position is allowed for the output type
+	 * 
+	 * @param pos
+	 *            Position to check
+	 */
+	protected void checkFieldRange(int pos) {
+		try {
+			getClassAtPos(pos);
+		} catch (IndexOutOfBoundsException e) {
+			throw new RuntimeException("Selected field is out of range");
+
+		}
+	}
+
 	/**
 	 * Creates a new {@link MergedDataStream} by merging {@link DataStream}
 	 * outputs of the same type with each other. The DataStreams merged using
@@ -483,50 +520,82 @@ public abstract class DataStream<OUT> {
 	}
 
 	/**
-	 * Applies an aggregation that sums the data stream at the given
-	 * position.
+	 * Applies an aggregation that sums the data stream at the given position.
 	 * 
 	 * @param positionToSum
 	 *            The position in the data point to sum
 	 * @return The transformed DataStream.
 	 */
+	@SuppressWarnings("unchecked")
 	public SingleOutputStreamOperator<OUT, ?> sum(int positionToSum) {
-		return aggregateAll(new StreamingSumAggregationFunction<OUT>(positionToSum));
+		checkFieldRange(positionToSum);
+		return aggregateAll((AggregationFunction<OUT>) SumAggregationFunction
+				.getSumFunction(positionToSum, getClassAtPos(positionToSum)));
+	}
+
+	/**
+	 * Applies an aggregation that sums the data stream at the first position .
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> sum() {
+		return sum(0);
 	}
-	
+
 	/**
-	 * Applies an aggregation that that gives the minimum of the data stream at the given
-	 * position.
+	 * Applies an aggregation that that gives the minimum of the data stream at
+	 * the given position.
 	 * 
 	 * @param positionToMin
 	 *            The position in the data point to minimize
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> min(int positionToMin) {
-		return aggregateAll(new StreamingMinAggregationFunction<OUT>(positionToMin));
+		checkFieldRange(positionToMin);
+		return aggregateAll(new MinAggregationFunction<OUT>(positionToMin));
 	}
-	
+
 	/**
-	 * Applies an aggregation that gives the maximum of the data stream at the given
-	 * position.
+	 * Applies an aggregation that that gives the minimum of the data stream at
+	 * the first position.
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min() {
+		return min(0);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum of the data stream at the
+	 * given position.
 	 * 
 	 * @param positionToMax
 	 *            The position in the data point to maximize
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> max(int positionToMax) {
-		return aggregateAll(new StreamingMaxAggregationFunction<OUT>(positionToMax));
+		checkFieldRange(positionToMax);
+		return aggregateAll(new MaxAggregationFunction<OUT>(positionToMax));
 	}
 
-	private SingleOutputStreamOperator<OUT, ?> aggregateAll(StreamingAggregationFunction<OUT> aggregate) {
+	/**
+	 * Applies an aggregation that gives the maximum of the data stream at the
+	 * first position.
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max() {
+		return max(0);
+	}
+
+	private SingleOutputStreamOperator<OUT, ?> aggregateAll(
+			AggregationFunction<OUT> aggregate) {
 		return aggregate(aggregate, new StreamReduceInvokable<OUT>(aggregate), "reduce");
 	}
-	
-	SingleOutputStreamOperator<OUT, ?> aggregate(StreamingAggregationFunction<OUT> aggregate, StreamReduceInvokable<OUT> invokable, String functionName) {
-		DataStream<OUT> inputStream = this.copy();
-		TypeInformation<?> info = this.jobGraphBuilder.getOutTypeInfo(inputStream.getId());
 
-		aggregate.setType(info);
+	SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate,
+			StreamReduceInvokable<OUT> invokable, String functionName) {
+		DataStream<OUT> inputStream = this.copy();
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = inputStream.addFunction(functionName,
 				aggregate, null, null, invokable);
@@ -1014,7 +1083,8 @@ public abstract class DataStream<OUT> {
 
 	private DataStreamSink<OUT> addSink(DataStream<OUT> inputStream,
 			SinkFunction<OUT> sinkFunction, TypeSerializerWrapper<OUT> typeWrapper) {
-		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink", outTypeWrapper);
+		DataStreamSink<OUT> returnStream = new DataStreamSink<OUT>(environment, "sink",
+				outTypeWrapper);
 
 		try {
 			jobGraphBuilder.addSink(returnStream.getId(), new SinkInvokable<OUT>(sinkFunction),

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/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 2e1ed57..06bec0a 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
@@ -21,10 +21,10 @@ import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
+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.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
@@ -52,14 +52,19 @@ public class GroupedDataStream<OUT> {
 	}
 
 	/**
-	 * Gets the output type.
+	 * Applies a reduce transformation on the grouped data stream grouped by the
+	 * given key position. The {@link ReduceFunction} will receive input values
+	 * based on the key value. Only input values with the same key will go to
+	 * the same reducer.The user can also extend {@link RichReduceFunction} to
+	 * gain access to other features provided by the {@link RichFuntion}
+	 * interface. Gets the output type.
 	 * 
 	 * @return The output type.
 	 */
 	public TypeInformation<OUT> getOutputType() {
 		return dataStream.getOutputType();
 	}
-	
+
 	/**
 	 * Applies a reduce transformation on the grouped data stream grouped on by
 	 * the given key position. The {@link ReduceFunction} will receive input
@@ -78,7 +83,7 @@ public class GroupedDataStream<OUT> {
 				ReduceFunction.class, 0), new FunctionTypeWrapper<OUT>(reducer,
 				ReduceFunction.class, 0), new GroupReduceInvokable<OUT>(reducer, keyPosition));
 	}
-	
+
 	/**
 	 * Applies a group reduce transformation on preset chunks of the grouped
 	 * data stream. The {@link GroupReduceFunction} will receive input values
@@ -214,8 +219,22 @@ public class GroupedDataStream<OUT> {
 	 *            The position in the data point to sum
 	 * @return The transformed DataStream.
 	 */
+	@SuppressWarnings("unchecked")
 	public SingleOutputStreamOperator<OUT, ?> sum(final int positionToSum) {
-		return aggregateGroup(new StreamingSumAggregationFunction<OUT>(positionToSum));
+		dataStream.checkFieldRange(positionToSum);
+		return aggregateGroup((AggregationFunction<OUT>) SumAggregationFunction
+				.getSumFunction(positionToSum, dataStream.getClassAtPos(positionToSum)));
+	}
+
+	/**
+	 * Applies an aggregation that sums the grouped data stream at the first
+	 * position, grouped by the given key position. Input values with the same
+	 * key will be summed.
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> sum() {
+		return sum(0);
 	}
 
 	/**
@@ -228,7 +247,21 @@ public class GroupedDataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> min(final int positionToMin) {
-		return aggregateGroup(new StreamingMinAggregationFunction<OUT>(positionToMin));
+		dataStream.checkFieldRange(positionToMin);
+		return aggregateGroup(new MinAggregationFunction<OUT>(positionToMin));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum of the grouped data stream
+	 * at the first position, grouped by the given key position. Input values
+	 * with the same key will be minimized.
+	 * 
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min() {
+		return min(0);
 	}
 
 	/**
@@ -241,10 +274,24 @@ public class GroupedDataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> max(final int positionToMax) {
-		return aggregateGroup(new StreamingMaxAggregationFunction<OUT>(positionToMax));
+		dataStream.checkFieldRange(positionToMax);
+		return aggregateGroup(new MaxAggregationFunction<OUT>(positionToMax));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum of the grouped data stream
+	 * at the first position, grouped by the given key position. Input values
+	 * with the same key will be maximized.
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max() {
+		return max(0);
 	}
 
-	private SingleOutputStreamOperator<OUT, ?> aggregateGroup(StreamingAggregationFunction<OUT> aggregate) {
-		return this.dataStream.aggregate(aggregate, new GroupReduceInvokable<OUT>(aggregate, keyPosition), "groupReduce");
+	private SingleOutputStreamOperator<OUT, ?> aggregateGroup(
+			AggregationFunction<OUT> aggregate) {
+		return this.dataStream.aggregate(aggregate, new GroupReduceInvokable<OUT>(aggregate,
+				keyPosition), "groupReduce");
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java
new file mode 100644
index 0000000..daae0b8
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java
@@ -0,0 +1,34 @@
+/**
+ * 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.common.functions.ReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+
+public abstract class AggregationFunction<T> implements ReduceFunction<T> {
+	private static final long serialVersionUID = 1L;
+	
+	public int position;
+	protected Tuple returnTuple;
+
+	public AggregationFunction(int pos) {
+		this.position = pos;
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
index dc74715..0819340 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.function.aggregation;
 
 import org.apache.flink.api.java.tuple.Tuple;
 
-public abstract class ComparableAggregationFunction<T> extends StreamingAggregationFunction<T> {
+public abstract class ComparableAggregationFunction<T> extends AggregationFunction<T> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -39,25 +39,25 @@ public abstract class ComparableAggregationFunction<T> extends StreamingAggregat
 			return (T) returnTuple;
 		} else if (value1 instanceof Comparable) {
 			if (isExtremal((Comparable<Object>) value1, value2)) {
-				value2 = value1;
+				return value1;
+			}else{
+				return value2;
 			}
 		} else {
 			throw new RuntimeException("The values " + value1 +  " and "+ value2 + " cannot be compared.");
 		}
-
-		return null;
 	}
 
 	public <R> void compare(Tuple tuple1, Tuple tuple2) throws InstantiationException,
 			IllegalAccessException {
-		copyTuple(tuple2);
 
 		Comparable<R> o1 = tuple1.getField(position);
 		R o2 = tuple2.getField(position);
 
 		if (isExtremal(o1, o2)) {
-			returnTuple.setField(o1, position);
+			tuple2.setField(o1, position);
 		}
+		returnTuple = tuple2;
 	}
 
 	public abstract <R> boolean isExtremal(Comparable<R> o1, R o2);

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

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java
deleted file mode 100644
index 42c1053..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.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.function.aggregation;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.types.TypeInformation;
-
-public abstract class StreamingAggregationFunction<T> implements ReduceFunction<T> {
-	private static final long serialVersionUID = 1L;
-	
-	protected int position;
-	private TypeSerializer<Tuple> typeSerializer;
-	protected Tuple returnTuple;
-
-	public StreamingAggregationFunction(int pos) {
-		this.position = pos;
-	}
-
-	@SuppressWarnings("unchecked")
-	public void setType(TypeInformation<?> type) {
-		this.typeSerializer = (TypeSerializer<Tuple>) type.createSerializer();
-	}
-
-	protected void copyTuple(Tuple tuple) throws InstantiationException, IllegalAccessException {
-		returnTuple = (Tuple) typeSerializer.createInstance();
-		typeSerializer.copy(tuple, returnTuple);
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
deleted file mode 100644
index 1a043c1..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
+++ /dev/null
@@ -1,64 +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.function.aggregation;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-public class StreamingSumAggregationFunction<T> extends StreamingAggregationFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	public StreamingSumAggregationFunction(int pos) {
-		super(pos);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public T reduce(T value1, T value2) throws Exception {
-		if (value1 instanceof Tuple) {
-			Tuple tuple1 = (Tuple) value1;
-			Tuple tuple2 = (Tuple) value2;
-
-			copyTuple(tuple2);
-			returnTuple.setField(add(tuple1.getField(position), tuple2.getField(position)), position);
-
-			return (T) returnTuple;
-		} else {
-			return (T) add(value1, value2);
-		}
-	}
-
-	private Object add(Object value1, Object value2) {
-		if (value1 instanceof Integer) {
-			return (Integer) value1 + (Integer) value2;
-		} else if (value1 instanceof Double) {
-			return (Double) value1 + (Double) value2;
-		} else if (value1 instanceof Float) {
-			return (Float) value1 + (Float) value2;
-		} else if (value1 instanceof Long) {
-			return (Long) value1 + (Long) value2;
-		} else if (value1 instanceof Short) {
-			return (short) ((Short) value1 + (Short) value2);
-		} else if (value1 instanceof Byte) {
-			return (byte) ((Byte) value1 + (Byte) value2);
-		} else {
-			throw new RuntimeException("DataStream cannot be summed because the class "
-					+ value1.getClass().getSimpleName() + " does not support the + operator.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregationFunction.java
new file mode 100644
index 0000000..3f54590
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregationFunction.java
@@ -0,0 +1,150 @@
+/**
+ * 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 abstract class SumAggregationFunction<T> extends AggregationFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	public SumAggregationFunction(int pos) {
+		super(pos);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public T reduce(T value1, T value2) throws Exception {
+		if (value1 instanceof Tuple) {
+			Tuple tuple1 = (Tuple) value1;
+			Tuple tuple2 = (Tuple) value2;
+
+			returnTuple = tuple2;
+			returnTuple.setField(add(tuple1.getField(position), tuple2.getField(position)),
+					position);
+
+			return (T) returnTuple;
+		} else {
+			return (T) add(value1, value2);
+		}
+	}
+
+	protected abstract Object add(Object value1, Object value2);
+
+	@SuppressWarnings("rawtypes")
+	public static <T> SumAggregationFunction getSumFunction(int pos, Class<T> type) {
+
+		if (type == Integer.class) {
+			return new IntSum<T>(pos);
+		} else if (type == Long.class) {
+			return new LongSum<T>(pos);
+		} else if (type == Short.class) {
+			return new ShortSum<T>(pos);
+		} else if (type == Double.class) {
+			return new DoubleSum<T>(pos);
+		} else if (type == Float.class) {
+			return new FloatSum<T>(pos);
+		} else if (type == Byte.class) {
+			return new ByteSum<T>(pos);
+		} else {
+			throw new RuntimeException("DataStream cannot be summed because the class "
+					+ type.getSimpleName() + " does not support the + operator.");
+		}
+
+	}
+
+	private static class IntSum<T> extends SumAggregationFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public IntSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Integer) value1 + (Integer) value2;
+		}
+	}
+
+	private static class LongSum<T> extends SumAggregationFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public LongSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Long) value1 + (Long) value2;
+		}
+	}
+
+	private static class DoubleSum<T> extends SumAggregationFunction<T> {
+
+		private static final long serialVersionUID = 1L;
+
+		public DoubleSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Double) value1 + (Double) value2;
+		}
+	}
+
+	private static class ShortSum<T> extends SumAggregationFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public ShortSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Short) value1 + (Short) value2;
+		}
+	}
+
+	private static class FloatSum<T> extends SumAggregationFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public FloatSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Float) value1 + (Float) value2;
+		}
+	}
+
+	private static class ByteSum<T> extends SumAggregationFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public ByteSum(int pos) {
+			super(pos);
+		}
+
+		@Override
+		protected Object add(Object value1, Object value2) {
+			return (Byte) value1 + (Byte) value2;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/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 95b3249..be6392e 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
@@ -30,7 +30,6 @@ public class BatchGroupReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN,
 	private static final long serialVersionUID = 1L;
 
 	int keyPosition;
-	protected GroupReduceFunction<IN, OUT> reducer;
 	private Iterator<StreamRecord<IN>> iterator;
 	private MutableTableState<Object, List<IN>> values;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
index dc27da4..05f888f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchIterator.java
@@ -17,8 +17,9 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
+import java.io.Serializable;
 import java.util.Iterator;
 
-public interface BatchIterator<IN> extends Iterator<IN> {
+public interface BatchIterator<IN> extends Iterator<IN>, Serializable {
 	public void reset();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
index 8b658f3..87b00f9 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
@@ -29,7 +29,6 @@ import org.apache.flink.streaming.state.MutableTableState;
 public class WindowGroupReduceInvokable<IN, OUT> extends WindowReduceInvokable<IN, OUT> {
 
 	int keyPosition;
-	protected GroupReduceFunction<IN, OUT> reducer;
 	private Iterator<StreamRecord<IN>> iterator;
 	private MutableTableState<Object, List<IN>> values;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/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 7a502aa..3861aab 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
@@ -18,15 +18,16 @@
 package org.apache.flink.streaming.api;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
-import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
+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.MinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.SumAggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
 import org.apache.flink.streaming.util.MockInvokable;
@@ -36,26 +37,33 @@ public class AggregationFunctionTest {
 
 	@Test
 	public void groupSumIntegerTest() {
-		ArrayList<Tuple2<Integer, Integer>> inputList = new ArrayList<Tuple2<Integer, Integer>>();
 
 		List<Tuple2<Integer, Integer>> expectedSumList = new ArrayList<Tuple2<Integer, Integer>>();
 		List<Tuple2<Integer, Integer>> expectedMinList = new ArrayList<Tuple2<Integer, Integer>>();
 		List<Tuple2<Integer, Integer>> expectedMaxList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Integer> expectedSumList0 = new ArrayList<Integer>();
+		List<Integer> expectedMinList0 = new ArrayList<Integer>();
+		List<Integer> expectedMaxList0 = new ArrayList<Integer>();
 		List<Tuple2<Integer, Integer>> expectedGroupSumList = new ArrayList<Tuple2<Integer, Integer>>();
 		List<Tuple2<Integer, Integer>> expectedGroupMinList = new ArrayList<Tuple2<Integer, Integer>>();
 		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;
 
 		for (int i = 0; i < 9; i++) {
-			inputList.add(new Tuple2<Integer, Integer>(i % 3, i));
-
+			simpleInput.add(i);
 			expectedSumList.add(new Tuple2<Integer, Integer>(i % 3, (i + 1) * i / 2));
 			expectedMinList.add(new Tuple2<Integer, Integer>(i % 3, 0));
 			expectedMaxList.add(new Tuple2<Integer, Integer>(i % 3, i));
 
+			expectedSumList0.add((i + 1) * i / 2);
+			expectedMaxList0.add(i);
+			expectedMinList0.add(0);
+
 			int groupedSum;
 			switch (i % 3) {
 			case 0:
@@ -74,30 +82,38 @@ public class AggregationFunctionTest {
 			expectedGroupMaxList.add(new Tuple2<Integer, Integer>(i % 3, i));
 		}
 
-		StreamingSumAggregationFunction<Tuple2<Integer, Integer>> sumFunction = new StreamingSumAggregationFunction<Tuple2<Integer, Integer>>(
-				1);
-		StreamingMinAggregationFunction<Tuple2<Integer, Integer>> minFunction = new StreamingMinAggregationFunction<Tuple2<Integer, Integer>>(
+		@SuppressWarnings("unchecked")
+		SumAggregationFunction<Tuple2<Integer, Integer>> sumFunction = SumAggregationFunction
+				.getSumFunction(1, Integer.class);
+		@SuppressWarnings("unchecked")
+		SumAggregationFunction<Integer> sumFunction0 = SumAggregationFunction
+				.getSumFunction(0, Integer.class);
+		MinAggregationFunction<Tuple2<Integer, Integer>> minFunction = new MinAggregationFunction<Tuple2<Integer, Integer>>(
 				1);
-		StreamingMaxAggregationFunction<Tuple2<Integer, Integer>> maxFunction = new StreamingMaxAggregationFunction<Tuple2<Integer, Integer>>(
+		MinAggregationFunction<Integer> minFunction0 = new MinAggregationFunction<Integer>(
+				0);
+		MaxAggregationFunction<Tuple2<Integer, Integer>> maxFunction = new MaxAggregationFunction<Tuple2<Integer, Integer>>(
 				1);
-
-		sumFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
-		minFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
-		maxFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
+		MaxAggregationFunction<Integer> maxFunction0 = new MaxAggregationFunction<Integer>(
+				0);
 
 		List<Tuple2<Integer, Integer>> sumList = MockInvokable.createAndExecute(
-				new StreamReduceInvokable<Tuple2<Integer, Integer>>(sumFunction), inputList);
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(sumFunction), getInputList());
+
 		List<Tuple2<Integer, Integer>> minList = MockInvokable.createAndExecute(
-				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minFunction), inputList);
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minFunction), getInputList());
+
 		List<Tuple2<Integer, Integer>> maxList = MockInvokable.createAndExecute(
-				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxFunction), inputList);
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxFunction), getInputList());
 
 		List<Tuple2<Integer, Integer>> groupedSumList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), inputList);
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), getInputList());
+
 		List<Tuple2<Integer, Integer>> groupedMinList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), inputList);
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), getInputList());
+
 		List<Tuple2<Integer, Integer>> groupedMaxList = MockInvokable.createAndExecute(
-				new GroupReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), inputList);
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), getInputList());
 
 		assertEquals(expectedSumList, sumList);
 		assertEquals(expectedMinList, minList);
@@ -105,5 +121,39 @@ 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 ));
+
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+		try {
+			env.generateSequence(1, 100).min(1);
+			fail();
+		} catch (Exception e) {
+			//Nothing to do here
+		}
+		try {
+			env.generateSequence(1, 100).min(2);
+			fail();
+		} catch (Exception e) {
+			//Nothing to do here
+		}
+		try {
+			env.generateSequence(1, 100).min(3);
+			fail();
+		} catch (Exception e) {
+			//Nothing to do here
+		}
+
+	}
+
+	private List<Tuple2<Integer, Integer>> getInputList() {
+		ArrayList<Tuple2<Integer, Integer>> inputList = new ArrayList<Tuple2<Integer, Integer>>();
+		for (int i = 0; i < 9; i++) {
+			inputList.add(new Tuple2<Integer, Integer>(i % 3, i));
+		}
+		return inputList;
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/bcbebed0/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
index 0c4bad1..acf2f28 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoFlatMapTest.java
@@ -80,11 +80,5 @@ public class CoFlatMapTest implements Serializable {
 			// expected
 		}
 		
-		try {
-			env.fromElements(10, 11).connect(ds2);
-			fail();
-		} catch (RuntimeException e) {
-			// expected
-		}
 	}
 }


[10/18] git commit: [streaming] DataStream type refactor for easier future extensions

Posted by mb...@apache.org.
[streaming] DataStream type refactor for easier future extensions


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

Branch: refs/heads/master
Commit: 9cbd68a8430cb4da4e63275974a3626d596896c2
Parents: 13a9277
Author: gyfora <gy...@gmail.com>
Authored: Sat Sep 6 18:14:07 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |  82 +++++++++------
 .../api/datastream/GroupedDataStream.java       | 105 +++++++------------
 .../api/datastream/IterativeDataStream.java     |  14 +--
 .../api/datastream/MergedDataStream.java        |  96 -----------------
 .../state/SlidingWindowStateIterator.java       |   2 +
 5 files changed, 89 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9cbd68a8/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 d78ceae..bd5b83c 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
@@ -81,7 +81,7 @@ import org.apache.flink.types.TypeInformation;
  *            The type of the DataStream, i.e., the type of the elements of the
  *            DataStream.
  */
-public abstract class DataStream<OUT> {
+public class DataStream<OUT> {
 
 	protected static Integer counter = 0;
 	protected final StreamExecutionEnvironment environment;
@@ -91,6 +91,7 @@ public abstract class DataStream<OUT> {
 	protected boolean selectAll;
 	protected StreamPartitioner<OUT> partitioner;
 	protected TypeSerializerWrapper<OUT> outTypeWrapper;
+	protected List<DataStream<OUT>> mergedStreams;
 
 	protected final JobGraphBuilder jobGraphBuilder;
 
@@ -120,6 +121,8 @@ public abstract class DataStream<OUT> {
 		this.selectAll = false;
 		this.partitioner = new ForwardPartitioner<OUT>();
 		this.outTypeWrapper = outTypeWrapper;
+		this.mergedStreams = new ArrayList<DataStream<OUT>>();
+		this.mergedStreams.add(this);
 	}
 
 	/**
@@ -137,6 +140,14 @@ public abstract class DataStream<OUT> {
 		this.partitioner = dataStream.partitioner;
 		this.jobGraphBuilder = dataStream.jobGraphBuilder;
 		this.outTypeWrapper = dataStream.outTypeWrapper;
+		this.mergedStreams = new ArrayList<DataStream<OUT>>();
+		this.mergedStreams.add(this);
+		if (dataStream.mergedStreams.size() > 1) {
+			for (int i = 1; i < dataStream.mergedStreams.size(); i++) {
+				this.mergedStreams.add(new DataStream<OUT>(dataStream.mergedStreams.get(i)));
+			}
+		}
+
 	}
 
 	/**
@@ -218,15 +229,26 @@ public abstract class DataStream<OUT> {
 	 *            The DataStreams to merge output with.
 	 * @return The {@link MergedDataStream}.
 	 */
-	public MergedDataStream<OUT> merge(DataStream<OUT>... streams) {
-		MergedDataStream<OUT> returnStream = new MergedDataStream<OUT>(this);
+	public DataStream<OUT> merge(DataStream<OUT>... streams) {
+		DataStream<OUT> returnStream = this.copy();
 
 		for (DataStream<OUT> stream : streams) {
-			returnStream.addConnection(stream);
+			for (DataStream<OUT> ds : stream.mergedStreams) {
+				validateMerge(ds.getId());
+				returnStream.mergedStreams.add(ds.copy());
+			}
 		}
 		return returnStream;
 	}
 
+	private void validateMerge(String id) {
+		for (DataStream<OUT> ds : this.mergedStreams) {
+			if (ds.getId().equals(id)) {
+				throw new RuntimeException("A DataStream cannot be merged with itself");
+			}
+		}
+	}
+
 	/**
 	 * Creates a new {@link ConnectedDataStream} by connecting
 	 * {@link DataStream} outputs of different type with each other. The
@@ -529,12 +551,12 @@ public abstract class DataStream<OUT> {
 	@SuppressWarnings("unchecked")
 	public SingleOutputStreamOperator<OUT, ?> sum(int positionToSum) {
 		checkFieldRange(positionToSum);
-		return aggregateAll((AggregationFunction<OUT>) SumAggregationFunction
-				.getSumFunction(positionToSum, getClassAtPos(positionToSum)));
+		return aggregate((AggregationFunction<OUT>) SumAggregationFunction.getSumFunction(
+				positionToSum, getClassAtPos(positionToSum)));
 	}
 
 	/**
-	 * Applies an aggregation that sums the data stream at the first position .
+	 * Syntactic sugar for sum(0)
 	 * 
 	 * @return The transformed DataStream.
 	 */
@@ -552,12 +574,11 @@ public abstract class DataStream<OUT> {
 	 */
 	public SingleOutputStreamOperator<OUT, ?> min(int positionToMin) {
 		checkFieldRange(positionToMin);
-		return aggregateAll(new MinAggregationFunction<OUT>(positionToMin));
+		return aggregate(new MinAggregationFunction<OUT>(positionToMin));
 	}
 
 	/**
-	 * Applies an aggregation that that gives the minimum of the data stream at
-	 * the first position.
+	 * Syntactic sugar for min(0)
 	 * 
 	 * @return The transformed DataStream.
 	 */
@@ -575,12 +596,11 @@ public abstract class DataStream<OUT> {
 	 */
 	public SingleOutputStreamOperator<OUT, ?> max(int positionToMax) {
 		checkFieldRange(positionToMax);
-		return aggregateAll(new MaxAggregationFunction<OUT>(positionToMax));
+		return aggregate(new MaxAggregationFunction<OUT>(positionToMax));
 	}
 
 	/**
-	 * Applies an aggregation that gives the maximum of the data stream at the
-	 * first position.
+	 * Syntactic sugar for max(0)
 	 * 
 	 * @return The transformed DataStream.
 	 */
@@ -588,20 +608,14 @@ public abstract class DataStream<OUT> {
 		return max(0);
 	}
 
-	private SingleOutputStreamOperator<OUT, ?> aggregateAll(
-			AggregationFunction<OUT> aggregate) {
-		return aggregate(aggregate, new StreamReduceInvokable<OUT>(aggregate), "reduce");
-	}
+	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
 
-	SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate,
-			StreamReduceInvokable<OUT> invokable, String functionName) {
-		DataStream<OUT> inputStream = this.copy();
+		StreamReduceInvokable<OUT> invokable = new StreamReduceInvokable<OUT>(aggregate);
 
-		SingleOutputStreamOperator<OUT, ?> returnStream = inputStream.addFunction(functionName,
-				aggregate, null, null, invokable);
-
-		this.jobGraphBuilder.setTypeWrappersFrom(inputStream.getId(), returnStream.getId());
+		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("reduce", aggregate, null,
+				null, invokable);
 
+		this.jobGraphBuilder.setTypeWrappersFrom(getId(), returnStream.getId());
 		return returnStream;
 	}
 
@@ -1032,7 +1046,9 @@ public abstract class DataStream<OUT> {
 	protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
 		DataStream<OUT> returnStream = this.copy();
 
-		returnStream.partitioner = partitioner;
+		for (DataStream<OUT> stream : returnStream.mergedStreams) {
+			stream.partitioner = partitioner;
+		}
 
 		return returnStream;
 	}
@@ -1051,14 +1067,9 @@ public abstract class DataStream<OUT> {
 	 *            Number of the type (used at co-functions)
 	 */
 	protected <X> void connectGraph(DataStream<X> inputStream, String outputID, int typeNumber) {
-		if (inputStream instanceof MergedDataStream) {
-			for (DataStream<X> stream : ((MergedDataStream<X>) inputStream).mergedStreams) {
-				jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
-						inputStream.userDefinedNames, inputStream.selectAll);
-			}
-		} else {
-			jobGraphBuilder.setEdge(inputStream.getId(), outputID, inputStream.partitioner,
-					typeNumber, inputStream.userDefinedNames, inputStream.selectAll);
+		for (DataStream<X> stream : inputStream.mergedStreams) {
+			jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
+					inputStream.userDefinedNames, inputStream.selectAll);
 		}
 
 	}
@@ -1104,5 +1115,8 @@ public abstract class DataStream<OUT> {
 	 * 
 	 * @return The copy
 	 */
-	protected abstract DataStream<OUT> copy();
+	protected DataStream<OUT> copy(){
+		return new DataStream<OUT>(this);
+	}
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9cbd68a8/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 06bec0a..c5b010d 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
@@ -22,47 +22,35 @@ import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 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.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.util.DefaultTimestamp;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
-import org.apache.flink.types.TypeInformation;
 
 /**
- * A GroupedDataStream represents a data stream which has been partitioned by
- * the given key in the values. Operators like {@link #reduce},
- * {@link #batchReduce} etc. can be applied on the {@link GroupedDataStream}.
+ * A GroupedDataStream represents a {@link DataStream} which has been
+ * partitioned by the given key in the values. Operators like {@link #reduce},
+ * {@link #batchReduce} etc. can be applied on the {@link GroupedDataStream} to
+ * get additional functionality by the grouping.
  *
  * @param <OUT>
  *            The output type of the {@link GroupedDataStream}.
  */
-public class GroupedDataStream<OUT> {
+public class GroupedDataStream<OUT> extends DataStream<OUT> {
 
-	DataStream<OUT> dataStream;
 	int keyPosition;
 
 	protected GroupedDataStream(DataStream<OUT> dataStream, int keyPosition) {
-		this.dataStream = dataStream.partitionBy(keyPosition);
+		super(dataStream.partitionBy(keyPosition));
 		this.keyPosition = keyPosition;
 	}
 
-	/**
-	 * Applies a reduce transformation on the grouped data stream grouped by the
-	 * given key position. The {@link ReduceFunction} will receive input values
-	 * based on the key value. Only input values with the same key will go to
-	 * the same reducer.The user can also extend {@link RichReduceFunction} to
-	 * gain access to other features provided by the {@link RichFuntion}
-	 * interface. Gets the output type.
-	 * 
-	 * @return The output type.
-	 */
-	public TypeInformation<OUT> getOutputType() {
-		return dataStream.getOutputType();
+	protected GroupedDataStream(GroupedDataStream<OUT> dataStream) {
+		super(dataStream);
+		this.keyPosition = dataStream.keyPosition;
 	}
 
 	/**
@@ -79,7 +67,7 @@ public class GroupedDataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> reduce(ReduceFunction<OUT> reducer) {
-		return dataStream.addFunction("groupReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
+		return addFunction("groupReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				ReduceFunction.class, 0), new FunctionTypeWrapper<OUT>(reducer,
 				ReduceFunction.class, 0), new GroupReduceInvokable<OUT>(reducer, keyPosition));
 	}
@@ -129,7 +117,7 @@ public class GroupedDataStream<OUT> {
 	public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
 			long batchSize, long slideSize) {
 
-		return dataStream.addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
+		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
 				GroupReduceFunction.class, 1), new BatchGroupReduceInvokable<OUT, R>(reducer,
 				batchSize, slideSize, keyPosition));
@@ -204,7 +192,7 @@ public class GroupedDataStream<OUT> {
 	 */
 	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
 			long windowSize, long slideInterval, Timestamp<OUT> timestamp) {
-		return dataStream.addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
+		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
 				GroupReduceFunction.class, 1), new WindowGroupReduceInvokable<OUT, R>(reducer,
 				windowSize, slideInterval, keyPosition, timestamp));
@@ -219,22 +207,8 @@ public class GroupedDataStream<OUT> {
 	 *            The position in the data point to sum
 	 * @return The transformed DataStream.
 	 */
-	@SuppressWarnings("unchecked")
 	public SingleOutputStreamOperator<OUT, ?> sum(final int positionToSum) {
-		dataStream.checkFieldRange(positionToSum);
-		return aggregateGroup((AggregationFunction<OUT>) SumAggregationFunction
-				.getSumFunction(positionToSum, dataStream.getClassAtPos(positionToSum)));
-	}
-
-	/**
-	 * Applies an aggregation that sums the grouped data stream at the first
-	 * position, grouped by the given key position. Input values with the same
-	 * key will be summed.
-	 * 
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<OUT, ?> sum() {
-		return sum(0);
+		return super.sum(positionToSum);
 	}
 
 	/**
@@ -247,21 +221,7 @@ public class GroupedDataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> min(final int positionToMin) {
-		dataStream.checkFieldRange(positionToMin);
-		return aggregateGroup(new MinAggregationFunction<OUT>(positionToMin));
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum of the grouped data stream
-	 * at the first position, grouped by the given key position. Input values
-	 * with the same key will be minimized.
-	 * 
-	 * @param positionToMin
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<OUT, ?> min() {
-		return min(0);
+		return super.min(positionToMin);
 	}
 
 	/**
@@ -274,24 +234,29 @@ public class GroupedDataStream<OUT> {
 	 * @return The transformed DataStream.
 	 */
 	public SingleOutputStreamOperator<OUT, ?> max(final int positionToMax) {
-		dataStream.checkFieldRange(positionToMax);
-		return aggregateGroup(new MaxAggregationFunction<OUT>(positionToMax));
+		return super.max(positionToMax);
 	}
 
-	/**
-	 * Applies an aggregation that gives the maximum of the grouped data stream
-	 * at the first position, grouped by the given key position. Input values
-	 * with the same key will be maximized.
-	 * 
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<OUT, ?> max() {
-		return max(0);
+	@Override
+	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
+
+		GroupReduceInvokable<OUT> invokable = new GroupReduceInvokable<OUT>(aggregate, keyPosition);
+
+		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("groupReduce", aggregate,
+				null, null, invokable);
+
+		this.jobGraphBuilder.setTypeWrappersFrom(getId(), returnStream.getId());
+		return returnStream;
+	}
+
+	@Override
+	protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
+		System.out.println("Setting the partitioning after groupBy can affect the grouping");
+		return super.setConnectionType(partitioner);
 	}
 
-	private SingleOutputStreamOperator<OUT, ?> aggregateGroup(
-			AggregationFunction<OUT> aggregate) {
-		return this.dataStream.aggregate(aggregate, new GroupReduceInvokable<OUT>(aggregate,
-				keyPosition), "groupReduce");
+	@Override
+	protected GroupedDataStream<OUT> copy() {
+		return new GroupedDataStream<OUT>(this);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9cbd68a8/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 573dffd..1450ba6 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
@@ -89,16 +89,10 @@ public class IterativeDataStream<IN> extends
 
 		List<String> name = Arrays.asList(new String[] { iterationName });
 
-		if (iterationTail instanceof MergedDataStream) {
-			for (DataStream<IN> stream : ((MergedDataStream<IN>) iterationTail).mergedStreams) {
-				String inputID = stream.getId();
-				jobGraphBuilder.setEdge(inputID, returnStream.getId(),
-						new ForwardPartitioner<IN>(), 0, name, false);
-			}
-		} else {
-
-			jobGraphBuilder.setEdge(iterationTail.getId(), returnStream.getId(),
-					new ForwardPartitioner<IN>(), 0, name, false);
+		for (DataStream<IN> stream : iterationTail.mergedStreams) {
+			String inputID = stream.getId();
+			jobGraphBuilder.setEdge(inputID, returnStream.getId(), new ForwardPartitioner<IN>(), 0,
+					name, false);
 		}
 
 		return iterationTail;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9cbd68a8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
deleted file mode 100755
index c1618b2..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/MergedDataStream.java
+++ /dev/null
@@ -1,96 +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.datastream;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.serialization.TypeSerializerWrapper;
-
-/**
- * The MergedDataStream represents a DataStream which consists of merged outputs
- * of DataStreams of the same type. Operators applied on this will transform all
- * the merged outputs jointly.
- *
- * @param <OUT>
- *            Type of the output.
- */
-public class MergedDataStream<OUT> extends DataStream<OUT> {
-
-	protected List<DataStream<OUT>> mergedStreams;
-
-	protected MergedDataStream(StreamExecutionEnvironment environment, String operatorType,
-			TypeSerializerWrapper<OUT> outTypeWrapper) {
-		super(environment, operatorType, outTypeWrapper);
-		this.mergedStreams = new ArrayList<DataStream<OUT>>();
-		this.mergedStreams.add(this);
-	}
-
-	protected MergedDataStream(DataStream<OUT> dataStream) {
-		super(dataStream);
-		mergedStreams = new ArrayList<DataStream<OUT>>();
-		if (dataStream instanceof MergedDataStream) {
-			for (DataStream<OUT> stream : ((MergedDataStream<OUT>) dataStream).mergedStreams) {
-				mergedStreams.add(stream);
-			}
-		} else {
-			this.mergedStreams.add(this);
-		}
-
-	}
-
-	protected void addConnection(DataStream<OUT> stream) {
-		if (stream instanceof MergedDataStream) {
-			MergedDataStream<OUT> mStream = (MergedDataStream<OUT>) stream;
-			for (DataStream<OUT> ds : mStream.mergedStreams) {
-				validateMerge(ds.id);
-				this.mergedStreams.add(ds.copy());
-			}
-		} else {
-			validateMerge(stream.id);
-			this.mergedStreams.add(stream.copy());
-		}
-	}
-
-	private void validateMerge(String id) {
-		for (DataStream<OUT> ds : this.mergedStreams) {
-			if (ds.id.equals(id)) {
-				throw new RuntimeException("A DataStream cannot be merged with itself");
-			}
-		}
-	}
-
-	@Override
-	protected DataStream<OUT> setConnectionType(StreamPartitioner<OUT> partitioner) {
-		MergedDataStream<OUT> returnStream = this.copy();
-
-		for (DataStream<OUT> stream : returnStream.mergedStreams) {
-			stream.partitioner = partitioner;
-		}
-
-		return returnStream;
-	}
-
-	@Override
-	protected MergedDataStream<OUT> copy() {
-		return new MergedDataStream<OUT>(this);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9cbd68a8/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
index 1a88095..bed3613 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowStateIterator.java
@@ -27,6 +27,8 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
 public class SlidingWindowStateIterator<T> implements BatchIterator<T> {
 
+	private static final long serialVersionUID = 1L;
+
 	private CircularFifoBuffer buffer;
 	// private StreamRecord<T> nextElement;
 


[17/18] [streaming] Updated logging to utilize SLF4J

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParser.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParser.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParser.java
deleted file mode 100644
index 9ecc4df..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParser.java
+++ /dev/null
@@ -1,175 +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.function;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.StringTokenizer;
-
-import org.apache.sling.commons.json.JSONArray;
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-
-/**
- * A JSONParser contains a JSONObject and provides opportunity to access
- * embedded fields in JSON code.
- */
-public class JSONParser {
-
-	private JSONObject originalJO;
-	private String searchedfield;
-	private Object temp;
-
-	/**
-	 * Construct a JSONParser from a string. The string has to be a JSON code
-	 * from which we want to get a field.
-	 * 
-	 * @param jsonText
-	 *            A string which contains a JSON code. String representation of
-	 *            a JSON code.
-	 * @throws JSONException
-	 *             If there is a syntax error in the source string.
-	 */
-	public JSONParser(String jsonText) throws JSONException {
-		originalJO = new JSONObject(jsonText);
-	}
-
-	/**
-	 * 
-	 * Parse the JSON code passed to the constructor to find the given key.
-	 * 
-	 * @param key
-	 *            The key whose value is searched for.
-	 * @return A JSONObject which has only one field called "retValue" and the
-	 *         value associated to it is the searched value. The methods of
-	 *         JSONObject can be used to get the field value in a desired
-	 *         format.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	public JSONObject parse(String key) throws JSONException {
-		initializeParser(key);
-		parsing();
-		return putResultInJSONObj();
-	}
-
-	/**
-	 * Prepare the fields of the class for the parsing
-	 * 
-	 * @param key
-	 *            The key whose value is searched for.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void initializeParser(String key) throws JSONException {
-		searchedfield = key;
-		temp = new JSONObject(originalJO.toString());
-	}
-
-	/**
-	 * This function goes through the given field and calls the appropriate
-	 * functions to treat the units between the punctuation marks.
-	 * 
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void parsing() throws JSONException {
-		StringTokenizer st = new StringTokenizer(searchedfield, ".");
-		while (st.hasMoreTokens()) {
-			find(st.nextToken());
-		}
-	}
-
-	/**
-	 * Search for the next part of the field and update the state if it was
-	 * found.
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void find(String nextToken) throws JSONException {
-		if (endsWithBracket(nextToken)) {
-			treatAllBracket(nextToken);
-		} else {
-			temp = ((JSONObject) temp).get(nextToken);
-		}
-	}
-
-	/**
-	 * Determine whether the given string ends with a closing square bracket ']'
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @return True if the given string ends with a closing square bracket ']'
-	 *         and false otherwise.
-	 */
-	private boolean endsWithBracket(String nextToken) {
-		return nextToken.substring(nextToken.length() - 1).endsWith("]");
-	}
-
-	/**
-	 * Handle (multidimensional) arrays. Treat the square bracket pairs one
-	 * after the other if necessary.
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @throws JSONException
-	 *             If the searched element is not found.
-	 */
-	private void treatAllBracket(String nextToken) throws JSONException {
-		List<String> list = Arrays.asList(nextToken.split("\\["));
-		ListIterator<String> iter = list.listIterator();
-
-		temp = ((JSONObject) temp).get(iter.next());
-
-		while (iter.hasNext()) {
-			int index = Integer.parseInt(cutBracket(iter.next()));
-			temp = ((JSONArray) temp).get(index);
-		}
-	}
-
-	/**
-	 * Remove the last character of the string.
-	 * 
-	 * @param string
-	 *            String to modify.
-	 * @return The given string without the last character.
-	 */
-	private String cutBracket(String string) {
-		return string.substring(0, string.length() - 1);
-	}
-
-	/**
-	 * Save the result of the search into a JSONObject.
-	 * 
-	 * @return A special JSONObject which contain only one key. The value
-	 *         associated to this key is the result of the search.
-	 * @throws JSONException
-	 *             If there is a problem creating the JSONObject. (e.g. invalid
-	 *             syntax)
-	 */
-	private JSONObject putResultInJSONObj() throws JSONException {
-		JSONObject jo = new JSONObject();
-		jo.put("retValue", temp);
-		return jo;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest.java b/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest.java
deleted file mode 100644
index 37cda2d..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest.java
+++ /dev/null
@@ -1,73 +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.function;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class JSONParserTest {
-
-	private String jsonText;
-	private String searchedField;
-
-	public JSONParserTest(String text, String field) {
-		jsonText = text;
-		searchedField = field;
-	}
-
-	@Parameters
-	public static Collection<Object[]> initParameterList() {
-
-		Object[][] parameterList = new Object[][] { 
-				{ "{\"key\":\"value\"}", 							"key" },
-				{ "{\"key\":[\"value\"]}", 							"key[0]" },
-				{ "{\"key\":[{\"key\":\"value\"}]}", 				"key[0].key" },
-				{ "{\"key\":[{\"key\":[{\"key\":\"value\"}]}]}", 	"key[0].key[0].key"},
-				{ "{\"key\":[1,[{\"key\":\"value\"}]]}", 			"key[1][0].key" },
-				{ "{\"key\":[1,[[\"key\",2,\"value\"]]]}", 			"key[1][0][2]" },
-				{ "{\"key\":{\"key\":{\"otherKey\":\"wrongValue\",\"key\":\"value\"},\"otherKey\":\"wrongValue\"},\"otherKey\":\"wrongValue\"}" , "key.key.key"}
-				};
-
-		return Arrays.asList(parameterList);
-	}
-
-	@Test
-	public void test() {
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-			String expected = "{\"retValue\":\"value\"}";
-
-			assertTrue(expected.equals(jo.toString()));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest2.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest2.java b/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest2.java
deleted file mode 100644
index 49a72a0..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/examples/function/JSONParserTest2.java
+++ /dev/null
@@ -1,94 +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.function;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-import org.junit.Test;
-
-
-public class JSONParserTest2 {
-	
-	@Test
-	public void testGetBooleanFunction() {
-		String jsonText = "{\"key\":true}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertTrue(jo.getBoolean("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-	@Test
-	public void testGetDoubleFunction() {
-		double expected = 12345.12345;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getDouble("retValue"),0.000001);
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-	@Test
-	public void testGetIntFunction() {
-		int expected = 15;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getInt("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-
-	@Test
-	public void testGetLongFunction() {
-		long expected = 111111111111L;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getLong("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-}


[15/18] [streaming] Added CoBatchGroupReduceInvokable, CoWindowGroupReduceInvokable and grouped variants

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoStreamReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoStreamReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoStreamReduceTest.java
new file mode 100755
index 0000000..34a35c1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoStreamReduceTest.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.streaming.api.function.co.CoReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoStreamReduceInvokable;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.junit.Test;
+
+public class CoStreamReduceTest {
+
+	public static class MyCoReduceFunction implements
+			CoReduceFunction<Integer, String, Integer> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Integer reduce1(Integer value1, Integer value2) {
+			return value1 * value2;
+		}
+
+		@Override
+		public String reduce2(String value1, String value2) {
+			return value1 + value2;
+		}
+
+		@Override
+		public Integer map1(Integer value) {
+			return value;
+		}
+
+		@Override
+		public Integer map2(String value) {
+			return Integer.parseInt(value);
+		}
+
+	}
+
+	@Test
+	public void coStreamReduceTest() {
+
+		CoStreamReduceInvokable<Integer, String, Integer> coReduce = new CoStreamReduceInvokable<Integer, String, Integer>(
+				new MyCoReduceFunction());
+
+		List<Integer> expected1 = Arrays.asList(1, 9, 2, 99, 6, 998, 24);
+		List<Integer> result = MockCoInvokable.createAndExecute(coReduce,
+				Arrays.asList(1, 2, 3, 4), Arrays.asList("9", "9", "8"));
+
+		assertEquals(expected1, result);
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowGroupReduceTest.java
new file mode 100644
index 0000000..6b3c2f3
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowGroupReduceTest.java
@@ -0,0 +1,256 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.function.co.CoGroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.operator.co.CoWindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.util.TimeStamp;
+import org.apache.flink.streaming.util.MockCoInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class CoWindowGroupReduceTest {
+
+	
+	public static final class MyCoGroupReduceFunction1 implements
+			CoGroupReduceFunction<Long, Integer, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce1(Iterable<Long> values, Collector<String> out) throws Exception {
+			Long gather = 0L;
+			for (Long value : values) {
+				gather += value;
+			}
+			out.collect(gather.toString());
+		}
+
+		@SuppressWarnings("unused")
+		@Override
+		public void reduce2(Iterable<Integer> values, Collector<String> out) throws Exception {
+			Integer gather = 0;
+			for (Integer value : values) {
+				gather++;
+			}
+			out.collect(gather.toString());
+		}
+	}
+
+	public static final class MyCoGroupReduceFunction2 implements
+			CoGroupReduceFunction<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce1(Iterable<Tuple2<String, Integer>> values, Collector<String> out)
+				throws Exception {
+			String gather = "";
+			for (Tuple2<String, Integer> value : values) {
+				gather += value.f0;
+			}
+			out.collect(gather);
+		}
+
+		@Override
+		public void reduce2(Iterable<Tuple2<Integer, Integer>> values, Collector<String> out)
+				throws Exception {
+			Integer gather = 0;
+			for (Tuple2<Integer, Integer> value : values) {
+				gather += value.f0;
+			}
+			out.collect(gather.toString());
+		}
+	}
+
+	public static final class MyTimeStamp1 implements TimeStamp<Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Long value) {
+			return value;
+		}
+
+		@Override
+		public long getStartTime() {
+			return 0L;
+		}
+	}
+
+	public static final class MyTimeStamp2 implements TimeStamp<Integer> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Integer value) {
+			return value.longValue();
+		}
+
+		@Override
+		public long getStartTime() {
+			return 0L;
+		}
+	}
+
+	public static final class MyTimeStamp3 implements TimeStamp<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Tuple2<String, Integer> value) {
+			return value.f1.longValue();
+		}
+
+		@Override
+		public long getStartTime() {
+			return 0L;
+		}
+	}
+
+	public static final class MyTimeStamp4 implements TimeStamp<Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long getTimestamp(Tuple2<Integer, Integer> value) {
+			return value.f1.longValue();
+		}
+
+		@Override
+		public long getStartTime() {
+			return 0L;
+		}
+	}
+
+	@Test
+	public void coWindowGroupReduceTest1() {
+
+		List<Long> inputs1 = new ArrayList<Long>();
+		inputs1.add(0L);
+		inputs1.add(2L);
+		inputs1.add(2L);
+		inputs1.add(3L);
+		inputs1.add(4L);
+		inputs1.add(5L);
+		inputs1.add(6L);
+		inputs1.add(6L);
+		inputs1.add(6L);
+		inputs1.add(8L);
+		inputs1.add(14L);
+		inputs1.add(15L);
+		inputs1.add(15L);
+
+		List<Integer> inputs2 = new ArrayList<Integer>();
+		inputs2.add(0);
+		inputs2.add(0);
+		inputs2.add(5);
+		inputs2.add(7);
+		inputs2.add(7);
+		inputs2.add(7);
+		inputs2.add(8);
+		inputs2.add(8);
+		inputs2.add(8);
+		inputs2.add(14);
+		inputs2.add(14);
+		inputs2.add(15);
+		inputs2.add(16);
+
+		CoWindowGroupReduceInvokable<Long, Integer, String> invokable = new CoWindowGroupReduceInvokable<Long, Integer, String>(
+				new MyCoGroupReduceFunction1(), 3L, 4L, 2L, 2L, new MyTimeStamp1(),
+				new MyTimeStamp2());
+
+		List<String> expected = new ArrayList<String>();
+		expected.add("4");
+		expected.add("11");
+		expected.add("27");
+		expected.add("26");
+		expected.add("8");
+		expected.add("0");
+		expected.add("14");
+		expected.add("2");
+		expected.add("1");
+		expected.add("4");
+		expected.add("6");
+		expected.add("3");
+		expected.add("0");
+		expected.add("3");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+
+	@Test
+	public void coWindowGroupReduceTest2() {
+
+		List<Tuple2<String, Integer>> inputs1 = new ArrayList<Tuple2<String, Integer>>();
+		inputs1.add(new Tuple2<String, Integer>("I", 1));
+		inputs1.add(new Tuple2<String, Integer>("t", 2));
+		inputs1.add(new Tuple2<String, Integer>("i", 4));
+		inputs1.add(new Tuple2<String, Integer>("s", 5));
+		inputs1.add(new Tuple2<String, Integer>("a", 7));
+		inputs1.add(new Tuple2<String, Integer>("l", 7));
+		inputs1.add(new Tuple2<String, Integer>("l", 8));
+		inputs1.add(new Tuple2<String, Integer>("o", 10));
+		inputs1.add(new Tuple2<String, Integer>("k", 11));
+		inputs1.add(new Tuple2<String, Integer>("a", 11));
+		inputs1.add(new Tuple2<String, Integer>("y", 11));
+		inputs1.add(new Tuple2<String, Integer>("!", 11));
+		inputs1.add(new Tuple2<String, Integer>(" ", 12));
+
+		List<Tuple2<Integer, Integer>> inputs2 = new ArrayList<Tuple2<Integer, Integer>>();
+		inputs2.add(new Tuple2<Integer, Integer>(10, 1));
+		inputs2.add(new Tuple2<Integer, Integer>(10, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(20, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(30, 2));
+		inputs2.add(new Tuple2<Integer, Integer>(10, 3));
+		inputs2.add(new Tuple2<Integer, Integer>(30, 4));
+		inputs2.add(new Tuple2<Integer, Integer>(40, 5));
+		inputs2.add(new Tuple2<Integer, Integer>(30, 6));
+		inputs2.add(new Tuple2<Integer, Integer>(20, 7));
+		inputs2.add(new Tuple2<Integer, Integer>(20, 7));
+		inputs2.add(new Tuple2<Integer, Integer>(10, 7));
+		inputs2.add(new Tuple2<Integer, Integer>(10, 8));
+		inputs2.add(new Tuple2<Integer, Integer>(30, 9));
+		inputs2.add(new Tuple2<Integer, Integer>(30, 10));
+
+		CoWindowGroupReduceInvokable<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String> invokable = new CoWindowGroupReduceInvokable<Tuple2<String, Integer>, Tuple2<Integer, Integer>, String>(
+				new MyCoGroupReduceFunction2(), 3L, 3L, 3L, 2L, new MyTimeStamp3(),
+				new MyTimeStamp4());
+
+		List<String> expected = new ArrayList<String>();
+		expected.add("It");
+		expected.add("is");
+		expected.add("all");
+		expected.add("okay!");
+		expected.add("70");
+		expected.add("100");
+		expected.add("100");
+		expected.add("90");
+
+		List<String> actualList = MockCoInvokable.createAndExecute(invokable, inputs1, inputs2);
+		Collections.sort(expected);
+		Collections.sort(actualList);
+
+		assertEquals(expected, actualList);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokableTest.java
deleted file mode 100755
index ae28034..0000000
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupReduceInvokableTest.java
+++ /dev/null
@@ -1,54 +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.operator;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.util.MockInvokable;
-import org.junit.Test;
-
-public class GroupReduceInvokableTest {
-
-	private static class MyReducer implements ReduceFunction<Integer>{
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1+value2;
-		}
-		
-	}
-	
-	@Test
-	public void test() {
-		GroupReduceInvokable<Integer> invokable1 = new GroupReduceInvokable<Integer>(
-				new MyReducer(),0);
-
-		List<Integer> expected = Arrays.asList(1,2,2,4,3);
-		List<Integer> actual = MockInvokable.createAndExecute(invokable1,
-				Arrays.asList(1, 1, 2, 2, 3));
-
-		assertEquals(expected, actual);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b6ffdbad/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokableTest.java
new file mode 100755
index 0000000..318b001
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedReduceInvokableTest.java
@@ -0,0 +1,54 @@
+/**
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.junit.Test;
+
+public class GroupedReduceInvokableTest {
+
+	private static class MyReducer implements ReduceFunction<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Integer reduce(Integer value1, Integer value2) throws Exception {
+			return value1 + value2;
+		}
+
+	}
+
+	@Test
+	public void test() {
+		GroupedReduceInvokable<Integer> invokable1 = new GroupedReduceInvokable<Integer>(
+				new MyReducer(), 0);
+
+		List<Integer> expected = Arrays.asList(1, 2, 2, 4, 3);
+		List<Integer> actual = MockInvokable.createAndExecute(invokable1,
+				Arrays.asList(1, 1, 2, 2, 3));
+
+		assertEquals(expected, actual);
+	}
+
+}


[03/18] git commit: [streaming] Updated directed emit to not use output names when all outputs are selected

Posted by mb...@apache.org.
[streaming] Updated directed emit to not use output names when all outputs are selected


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

Branch: refs/heads/master
Commit: fbfcc9eb4d835df1a646b6f963527e86ef380ec9
Parents: 47dca69
Author: ghermann <re...@gmail.com>
Authored: Tue Sep 2 14:47:48 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    |   9 +-
 .../flink/streaming/api/StreamConfig.java       |  56 ++--
 .../api/collector/DirectedStreamCollector.java  | 224 ++++++++-------
 .../api/collector/StreamCollector.java          | 285 ++++++++++---------
 .../streaming/api/datastream/DataStream.java    |   7 +-
 .../api/datastream/IterativeDataStream.java     |   4 +-
 .../datastream/SingleOutputStreamOperator.java  |  18 +-
 .../api/datastream/SplitDataStream.java         |  20 +-
 .../api/streamcomponent/OutputHandler.java      |   5 +-
 .../api/collector/DirectedOutputTest.java       | 120 ++++----
 .../api/collector/StreamCollectorTest.java      |   2 +-
 11 files changed, 378 insertions(+), 372 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 aaa7161..7973324 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
@@ -64,6 +64,7 @@ public class JobGraphBuilder {
 	private Map<String, List<String>> outEdgeList;
 	private Map<String, List<Integer>> outEdgeType;
 	private Map<String, List<List<String>>> outEdgeNames;
+	private Map<String, List<Boolean>> outEdgeSelectAll;
 	private Map<String, Boolean> mutability;
 	private Map<String, List<String>> inEdgeList;
 	private Map<String, List<StreamPartitioner<?>>> connectionTypes;
@@ -106,6 +107,7 @@ public class JobGraphBuilder {
 		outEdgeList = new HashMap<String, List<String>>();
 		outEdgeType = new HashMap<String, List<Integer>>();
 		outEdgeNames = new HashMap<String, List<List<String>>>();
+		outEdgeSelectAll = new HashMap<String, List<Boolean>>();
 		mutability = new HashMap<String, Boolean>();
 		inEdgeList = new HashMap<String, List<String>>();
 		connectionTypes = new HashMap<String, List<StreamPartitioner<?>>>();
@@ -203,7 +205,7 @@ public class JobGraphBuilder {
 
 		setEdge(componentName, iterationHead,
 				connectionTypes.get(inEdgeList.get(iterationHead).get(0))
-						.get(0), 0, new ArrayList<String>());
+						.get(0), 0, new ArrayList<String>(), false);
 
 		iterationWaitTime.put(iterationIDtoSourceName.get(iterationID),
 				waitTime);
@@ -360,6 +362,7 @@ public class JobGraphBuilder {
 		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<?>>());
@@ -484,12 +487,13 @@ public class JobGraphBuilder {
 	public void setEdge(String upStreamComponentName,
 			String downStreamComponentName,
 			StreamPartitioner<?> partitionerObject, int typeNumber,
-			List<String> outputNames) {
+			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);
 	}
 
 	/**
@@ -541,6 +545,7 @@ public class JobGraphBuilder {
 
 		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));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 445020a..c2a4c21 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
@@ -38,6 +38,7 @@ public class StreamConfig {
 	private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs";
 	private static final String NUMBER_OF_INPUTS = "numberOfInputs";
 	private static final String OUTPUT_NAME = "outputName_";
+	private static final String OUTPUT_SELECT_ALL = "outputSelectAll_";
 	private static final String PARTITIONER_OBJECT = "partitionerObject_";
 	private static final String NUMBER_OF_OUTPUT_CHANNELS = "numOfOutputs_";
 	private static final String ITERATION_ID = "iteration-id";
@@ -149,8 +150,7 @@ public class StreamConfig {
 			config.setClass(USER_FUNCTION, invokableObject.getClass());
 
 			try {
-				config.setBytes(SERIALIZEDUDF,
-						SerializationUtils.serialize(invokableObject));
+				config.setBytes(SERIALIZEDUDF, SerializationUtils.serialize(invokableObject));
 			} catch (SerializationException e) {
 				throw new RuntimeException("Cannot serialize invokable object "
 						+ invokableObject.getClass(), e);
@@ -162,11 +162,10 @@ public class StreamConfig {
 		try {
 			return deserializeObject(config.getBytes(SERIALIZEDUDF, null));
 		} catch (Exception e) {
-			throw new StreamComponentException(
-					"Cannot instantiate user function", e);
+			throw new StreamComponentException("Cannot instantiate user function", e);
 		}
 	}
-	
+
 	public void setComponentName(String componentName) {
 		config.setString(COMPONENT_NAME, componentName);
 	}
@@ -184,8 +183,7 @@ public class StreamConfig {
 
 	public Object getFunction() {
 		try {
-			return SerializationUtils.deserialize(config.getBytes(FUNCTION,
-					null));
+			return SerializationUtils.deserialize(config.getBytes(FUNCTION, null));
 		} catch (SerializationException e) {
 			throw new RuntimeException("Cannot deserialize invokable object", e);
 		}
@@ -214,8 +212,8 @@ public class StreamConfig {
 		try {
 			return deserializeObject(config.getBytes(OUTPUT_SELECTOR, null));
 		} catch (Exception e) {
-			throw new StreamComponentException(
-					"Cannot deserialize and instantiate OutputSelector", e);
+			throw new StreamComponentException("Cannot deserialize and instantiate OutputSelector",
+					e);
 		}
 	}
 
@@ -235,30 +233,36 @@ public class StreamConfig {
 		return config.getLong(ITERATON_WAIT, 0);
 	}
 
-	public void setNumberOfOutputChannels(int outputIndex,
-			Integer numberOfOutputChannels) {
-		config.setInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex,
-				numberOfOutputChannels);
+	public void setNumberOfOutputChannels(int outputIndex, Integer numberOfOutputChannels) {
+		config.setInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, numberOfOutputChannels);
 	}
 
 	public int getNumberOfOutputChannels(int outputIndex) {
 		return config.getInteger(NUMBER_OF_OUTPUT_CHANNELS + outputIndex, 0);
 	}
 
-	public <T> void setPartitioner(int outputIndex,
-			StreamPartitioner<T> partitionerObject) {
+	public <T> void setPartitioner(int outputIndex, StreamPartitioner<T> partitionerObject) {
 
 		config.setBytes(PARTITIONER_OBJECT + outputIndex,
 				SerializationUtils.serialize(partitionerObject));
 	}
 
-	public <T> StreamPartitioner<T> getPartitioner(int outputIndex)
-			throws ClassNotFoundException, IOException {
-		return deserializeObject(config.getBytes(PARTITIONER_OBJECT
-				+ outputIndex,
+	public <T> StreamPartitioner<T> getPartitioner(int outputIndex) throws ClassNotFoundException,
+			IOException {
+		return deserializeObject(config.getBytes(PARTITIONER_OBJECT + outputIndex,
 				SerializationUtils.serialize(new ShufflePartitioner<T>())));
 	}
 
+	public void setSelectAll(int outputIndex, Boolean selectAll) {
+		if (selectAll != null) {
+			config.setBoolean(OUTPUT_SELECT_ALL + outputIndex, selectAll);
+		}
+	}
+
+	public boolean getSelectAll(int outputIndex) {
+		return config.getBoolean(OUTPUT_SELECT_ALL + outputIndex, false);
+	}
+
 	public void setOutputName(int outputIndex, List<String> outputName) {
 		if (outputName != null) {
 			config.setBytes(OUTPUT_NAME + outputIndex,
@@ -268,8 +272,8 @@ public class StreamConfig {
 
 	@SuppressWarnings("unchecked")
 	public List<String> getOutputName(int outputIndex) {
-		return (List<String>) SerializationUtils.deserialize(config.getBytes(
-				OUTPUT_NAME + outputIndex, null));
+		return (List<String>) SerializationUtils.deserialize(config.getBytes(OUTPUT_NAME
+				+ outputIndex, null));
 	}
 
 	public void setNumberOfInputs(int numberOfInputs) {
@@ -296,20 +300,18 @@ public class StreamConfig {
 		return config.getInteger(INPUT_TYPE + inputNumber, 0);
 	}
 
-	public void setFunctionClass(
-			Class<? extends AbstractRichFunction> functionClass) {
+	public void setFunctionClass(Class<? extends AbstractRichFunction> functionClass) {
 		config.setClass("functionClass", functionClass);
 	}
 
 	@SuppressWarnings("unchecked")
 	public Class<? extends AbstractRichFunction> getFunctionClass() {
-		return (Class<? extends AbstractRichFunction>) config.getClass(
-				"functionClass", null);
+		return (Class<? extends AbstractRichFunction>) config.getClass("functionClass", null);
 	}
 
 	@SuppressWarnings("unchecked")
-	protected static <T> T deserializeObject(byte[] serializedObject)
-			throws IOException, ClassNotFoundException {
+	protected static <T> T deserializeObject(byte[] serializedObject) throws IOException,
+			ClassNotFoundException {
 		return (T) SerializationUtils.deserialize(serializedObject);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index 54b1a98..ab6caea 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -1,105 +1,119 @@
-/**
- * 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.collector;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-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.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A StreamCollector that uses user defined output names and a user defined
- * output selector to make directed emits.
- * 
- * @param <OUT>
- *            Type of the Tuple collected.
- */
-public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
-
-	OutputSelector<OUT> outputSelector;
-	private static final Logger LOG = LoggerFactory.getLogger(DirectedStreamCollector.class);
-	private Set<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> emitted;
-
-	/**
-	 * Creates a new DirectedStreamCollector
-	 * 
-	 * @param channelID
-	 *            Channel ID of the Task
-	 * @param serializationDelegate
-	 *            Serialization delegate used for serialization
-	 * @param outputSelector
-	 *            User defined {@link OutputSelector}
-	 */
-	public DirectedStreamCollector(int channelID,
-			SerializationDelegate<StreamRecord<OUT>> serializationDelegate,
-			OutputSelector<OUT> outputSelector) {
-		super(channelID, serializationDelegate);
-		this.outputSelector = outputSelector;
-		this.emitted = new HashSet<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
-
-	}
-
-	/**
-	 * Collects and emits a tuple to the outputs by reusing a StreamRecord
-	 * object.
-	 * 
-	 * @param outputObject
-	 *            Object to be collected and emitted.
-	 */
-	@Override
-	public void collect(OUT outputObject) {
-		streamRecord.setObject(outputObject);
-		emit(streamRecord);
-	}
-
-	/**
-	 * Emits a StreamRecord to the outputs selected by the user defined
-	 * OutputSelector
-	 * 
-	 * @param streamRecord
-	 *            Record to emit.
-	 */
-	private void emit(StreamRecord<OUT> streamRecord) {
-		Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getObject());
-		streamRecord.newId(channelID);
-		serializationDelegate.setInstance(streamRecord);
-		emitted.clear();
-		for (String outputName : outputNames) {
-			try {
-				for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputMap
-						.get(outputName)) {
-					if (!emitted.contains(output)) {
-						output.emit(serializationDelegate);
-						emitted.add(output);
-					}
-				}
-			} catch (Exception e) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error(String.format("Emit to %s failed due to: %s", outputName,
-							StringUtils.stringifyException(e)));
-				}
-			}
-		}
-	}
-}
+/**
+ * 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.collector;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+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.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A StreamCollector that uses user defined output names and a user defined
+ * output selector to make directed emits.
+ * 
+ * @param <OUT>
+ *            Type of the Tuple collected.
+ */
+public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(DirectedStreamCollector.class);
+	
+	OutputSelector<OUT> outputSelector;
+	private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> selectAllOutputs;
+	private Set<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> emitted;
+
+	/**
+	 * Creates a new DirectedStreamCollector
+	 * 
+	 * @param channelID
+	 *            Channel ID of the Task
+	 * @param serializationDelegate
+	 *            Serialization delegate used for serialization
+	 * @param outputSelector
+	 *            User defined {@link OutputSelector}
+	 */
+	public DirectedStreamCollector(int channelID,
+			SerializationDelegate<StreamRecord<OUT>> serializationDelegate,
+			OutputSelector<OUT> outputSelector) {
+		super(channelID, serializationDelegate);
+		this.outputSelector = outputSelector;
+		this.emitted = new HashSet<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+		this.selectAllOutputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+	}
+
+	@Override
+	public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
+			List<String> outputNames, boolean isSelectAllOutput) {
+
+		if (isSelectAllOutput) {
+			selectAllOutputs.add(output);
+		} else {
+			addOneOutput(output, outputNames, isSelectAllOutput);
+		}
+	}
+
+	/**
+	 * Emits a StreamRecord to the outputs selected by the user defined
+	 * OutputSelector
+	 *
+	 */
+	protected void emitToOutputs() {
+		Collection<String> outputNames = outputSelector.getOutputs(streamRecord.getObject());
+		emitted.clear();
+		for (String outputName : outputNames) {
+			List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputList = outputMap
+					.get(outputName);
+			if (outputList == null) {
+				if (LOG.isErrorEnabled()) {
+					LOG.error(String.format(
+							"Cannot emit because no output is selected with the name: %s",
+							outputName));
+				}
+			}
+
+			try {
+				for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : selectAllOutputs) {
+					if (!emitted.contains(output)) {
+						output.emit(serializationDelegate);
+						emitted.add(output);
+					}
+				}
+
+				for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputList) {
+					if (!emitted.contains(output)) {
+						output.emit(serializationDelegate);
+						emitted.add(output);
+					}
+				}
+			} catch (Exception e) {
+				if (LOG.isErrorEnabled()) {
+					LOG.error(String.format("Emit to %s failed due to: %s", outputName,
+							StringUtils.stringifyException(e)));
+				}
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index c6ba1ef..ce4069e 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -1,137 +1,148 @@
-/**
- * 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.collector;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-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.util.Collector;
-import org.apache.flink.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Collector for tuples in Apache Flink stream processing. The collected values
- * will be wrapped with ID in a {@link StreamRecord} and then emitted to the
- * outputs.
- * 
- * @param <OUT>
- *            Type of the Tuples/Objects collected.
- */
-public class StreamCollector<OUT> implements Collector<OUT> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamCollector.class);
-
-	protected StreamRecord<OUT> streamRecord;
-	protected int channelID;
-	private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
-	protected Map<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>> outputMap;
-	protected SerializationDelegate<StreamRecord<OUT>> serializationDelegate;
-
-	/**
-	 * Creates a new StreamCollector
-	 * 
-	 * @param channelID
-	 *            Channel ID of the Task
-	 * @param serializationDelegate
-	 *            Serialization delegate used for serialization
-	 */
-	public StreamCollector(int channelID,
-			SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
-		this.serializationDelegate = serializationDelegate;
-		if (serializationDelegate != null) {
-			this.streamRecord = serializationDelegate.getInstance();
-		} else {
-			this.streamRecord = new StreamRecord<OUT>();
-		}
-		this.channelID = channelID;
-		this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
-		this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>>();
-	}
-
-	/**
-	 * Adds an output with the given user defined name
-	 * 
-	 * @param output
-	 *            The RecordWriter object representing the output.
-	 * @param outputNames
-	 *            User defined names of the output.
-	 */
-	public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
-			List<String> outputNames) {
-		outputs.add(output);
-		for (String outputName : outputNames) {
-			if (outputName != null) {
-				if (!outputMap.containsKey(outputName)) {
-					outputMap
-							.put(outputName,
-									new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>());
-					outputMap.get(outputName).add(output);
-				} else {
-					if (!outputMap.get(outputName).contains(output)) {
-						outputMap.get(outputName).add(output);
-					}
-				}
-
-			}
-		}
-	}
-
-	/**
-	 * Collects and emits a tuple/object to the outputs by reusing a
-	 * StreamRecord object.
-	 * 
-	 * @param outputObject
-	 *            Object to be collected and emitted.
-	 */
-	@Override
-	public void collect(OUT outputObject) {
-		streamRecord.setObject(outputObject);
-		emit(streamRecord);
-	}
-
-	/**
-	 * Emits a StreamRecord to all the outputs.
-	 * 
-	 * @param streamRecord
-	 *            StreamRecord to emit.
-	 */
-	private void emit(StreamRecord<OUT> streamRecord) {
-		streamRecord.newId(channelID);
-		serializationDelegate.setInstance(streamRecord);
-		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
-			try {
-				output.emit(serializationDelegate);
-			} catch (Exception e) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error(String.format("Emit failed due to: %s",
-							StringUtils.stringifyException(e)));
-				}
-			}
-		}
-	}
-
-	@Override
-	public void close() {
-	}
-}
+/**
+ * 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.collector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.util.Collector;
+import org.apache.flink.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Collector for tuples in Apache Flink stream processing. The collected values
+ * will be wrapped with ID in a {@link StreamRecord} and then emitted to the
+ * outputs.
+ * 
+ * @param <OUT>
+ *            Type of the Tuples/Objects collected.
+ */
+public class StreamCollector<OUT> implements Collector<OUT> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(StreamCollector.class);
+
+	protected StreamRecord<OUT> streamRecord;
+	protected int channelID;
+	protected List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputs;
+	protected Map<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>> outputMap;
+	protected SerializationDelegate<StreamRecord<OUT>> serializationDelegate;
+
+	/**
+	 * Creates a new StreamCollector
+	 * 
+	 * @param channelID
+	 *            Channel ID of the Task
+	 * @param serializationDelegate
+	 *            Serialization delegate used for serialization
+	 */
+	public StreamCollector(int channelID,
+			SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
+		this.serializationDelegate = serializationDelegate;
+		if (serializationDelegate != null) {
+			this.streamRecord = serializationDelegate.getInstance();
+		} else {
+			this.streamRecord = new StreamRecord<OUT>();
+		}
+		this.channelID = channelID;
+		this.outputs = new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>();
+		this.outputMap = new HashMap<String, List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>>();
+	}
+
+	/**
+	 * Adds an output with the given user defined name
+	 * 
+	 * @param output
+	 *            The RecordWriter object representing the output.
+	 * @param outputNames
+	 *            User defined names of the output.
+	 * @param isSelectAllOutput
+	 *            Marks whether all the outputs are selected.
+	 */
+	public void addOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
+			List<String> outputNames, boolean isSelectAllOutput) {
+		addOneOutput(output, outputNames, isSelectAllOutput);
+	}
+
+	protected void addOneOutput(RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output,
+			List<String> outputNames, boolean isSelectAllOutput) {
+		outputs.add(output);
+		for (String outputName : outputNames) {
+			if (outputName != null) {
+				if (!outputMap.containsKey(outputName)) {
+					outputMap
+							.put(outputName,
+									new ArrayList<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>>());
+					outputMap.get(outputName).add(output);
+				} else {
+					if (!outputMap.get(outputName).contains(output)) {
+						outputMap.get(outputName).add(output);
+					}
+				}
+
+			}
+		}
+	}
+	
+	/**
+	 * Collects and emits a tuple/object to the outputs by reusing a
+	 * StreamRecord object.
+	 * 
+	 * @param outputObject
+	 *            Object to be collected and emitted.
+	 */
+	@Override
+	public void collect(OUT outputObject) {
+		streamRecord.setObject(outputObject);
+		emit(streamRecord);
+	}
+
+	/**
+	 * Emits a StreamRecord to the outputs.
+	 * 
+	 * @param streamRecord
+	 *            StreamRecord to emit.
+	 */
+	private void emit(StreamRecord<OUT> streamRecord) {
+		streamRecord.newId(channelID);
+		serializationDelegate.setInstance(streamRecord);
+		emitToOutputs();
+	}
+	
+	protected void emitToOutputs() {
+		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
+			try {
+				output.emit(serializationDelegate);
+			} catch (Exception e) {
+				if (LOG.isErrorEnabled()) {
+					LOG.error(String.format("Emit failed due to: %s",
+							StringUtils.stringifyException(e)));
+				}
+			}
+		}
+	}
+
+	@Override
+	public void close() {
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 a2994dc..64a07b5 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
@@ -82,6 +82,7 @@ public abstract class DataStream<OUT> {
 	protected final String id;
 	protected int degreeOfParallelism;
 	protected List<String> userDefinedNames;
+	protected boolean selectAll;
 	protected StreamPartitioner<OUT> partitioner;
 
 	protected final JobGraphBuilder jobGraphBuilder;
@@ -106,6 +107,7 @@ public abstract class DataStream<OUT> {
 		this.degreeOfParallelism = environment.getDegreeOfParallelism();
 		this.jobGraphBuilder = environment.getJobGraphBuilder();
 		this.userDefinedNames = new ArrayList<String>();
+		this.selectAll = false;
 		this.partitioner = new ForwardPartitioner<OUT>();
 
 	}
@@ -121,6 +123,7 @@ public abstract class DataStream<OUT> {
 		this.id = dataStream.id;
 		this.degreeOfParallelism = dataStream.degreeOfParallelism;
 		this.userDefinedNames = new ArrayList<String>(dataStream.userDefinedNames);
+		this.selectAll = dataStream.selectAll;
 		this.partitioner = dataStream.partitioner;
 		this.jobGraphBuilder = dataStream.jobGraphBuilder;
 
@@ -911,11 +914,11 @@ public abstract class DataStream<OUT> {
 		if (inputStream instanceof MergedDataStream) {
 			for (DataStream<X> stream : ((MergedDataStream<X>) inputStream).mergedStreams) {
 				jobGraphBuilder.setEdge(stream.getId(), outputID, stream.partitioner, typeNumber,
-						inputStream.userDefinedNames);
+						inputStream.userDefinedNames, inputStream.selectAll);
 			}
 		} else {
 			jobGraphBuilder.setEdge(inputStream.getId(), outputID, inputStream.partitioner,
-					typeNumber, inputStream.userDefinedNames);
+					typeNumber, inputStream.userDefinedNames, inputStream.selectAll);
 		}
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 46be328..16362ba 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
@@ -93,12 +93,12 @@ public class IterativeDataStream<IN> extends
 			for (DataStream<IN> stream : ((MergedDataStream<IN>) iterationTail).mergedStreams) {
 				String inputID = stream.getId();
 				jobGraphBuilder.setEdge(inputID, returnStream.getId(),
-						new ForwardPartitioner<IN>(), 0, name);
+						new ForwardPartitioner<IN>(), 0, name, false);
 			}
 		} else {
 
 			jobGraphBuilder.setEdge(iterationTail.getId(), returnStream.getId(),
-					new ForwardPartitioner<IN>(), 0, name);
+					new ForwardPartitioner<IN>(), 0, name, false);
 		}
 
 		return iterationTail;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 b1b0939..1f01feb 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
@@ -104,22 +104,6 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 	 * @return The {@link SplitDataStream}
 	 */
 	public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector) {
-		return split(outputSelector, null);
-	}
-
-	/**
-	 * Operator used for directing tuples to specific named outputs using an
-	 * {@link OutputSelector}. Calling this method on an operator creates a new
-	 * {@link SplitDataStream}.
-	 * 
-	 * @param outputSelector
-	 *            The user defined {@link OutputSelector} for directing the
-	 *            tuples.
-	 * @param outputNames
-	 *            An array of all the output names to be used for selectAll
-	 * @return The {@link SplitDataStream}
-	 */
-	public SplitDataStream<OUT> split(OutputSelector<OUT> outputSelector, String[] outputNames) {
 		try {
 			jobGraphBuilder.setOutputSelector(id, SerializationUtils.serialize(outputSelector));
 
@@ -127,7 +111,7 @@ public class SingleOutputStreamOperator<OUT, O extends SingleOutputStreamOperato
 			throw new RuntimeException("Cannot serialize OutputSelector");
 		}
 
-		return new SplitDataStream<OUT>(this, outputNames);
+		return new SplitDataStream<OUT>(this);
 	}
 
 	@SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
index a6cf4b1..0ddb4f0 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
@@ -32,11 +32,9 @@ import org.apache.flink.streaming.api.collector.OutputSelector;
 public class SplitDataStream<OUT> {
 
 	DataStream<OUT> dataStream;
-	String[] allNames;
 
-	protected SplitDataStream(DataStream<OUT> dataStream, String[] outputNames) {
+	protected SplitDataStream(DataStream<OUT> dataStream) {
 		this.dataStream = dataStream.copy();
-		this.allNames = outputNames;
 	}
 
 	/**
@@ -52,22 +50,18 @@ public class SplitDataStream<OUT> {
 	}
 
 	/**
-	 * Selects all output names from a split data stream. Output names must
-	 * predefined to use selectAll.
+	 * Selects all output names from a split data stream.
 	 * 
 	 * @return Returns the selected DataStream
 	 */
-	public DataStream<OUT> selectAll() {
-		if (allNames != null) {
-			return selectOutput(allNames);
-		} else {
-			throw new RuntimeException(
-					"Output names must be predefined in order to use select all.");
-		}
+	public DataStream<OUT> selectAll() {
+		DataStream<OUT> returnStream = dataStream.copy();
+		returnStream.selectAll = true;
+		return returnStream;
 	}
 
 	private DataStream<OUT> selectOutput(String[] outputName) {
-		DataStream<OUT> returnStream = dataStream.copy();
+		DataStream<OUT> returnStream = dataStream.copy();
 		returnStream.userDefinedNames = Arrays.asList(outputName);
 		return returnStream;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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
index e19eeaa..76277dc 100644
--- 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
@@ -127,9 +127,10 @@ public class OutputHandler<OUT> {
 
 		outputs.add(output);
 		List<String> outputName = configuration.getOutputName(outputNumber);
-
+		boolean isSelectAllOutput = configuration.getSelectAll(outputNumber);
+		
 		if (collector != null) {
-			collector.addOutput(output, outputName);
+			collector.addOutput(output, outputName, isSelectAllOutput);
 		}
 
 		if (LOG.isTraceEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index 1608b7b..fdf9db3 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -1,30 +1,17 @@
-/**
- * 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.collector;
 
 import static org.junit.Assert.assertEquals;
 
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
-import org.apache.flink.api.java.functions.RichMapFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.streaming.api.collector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -33,74 +20,79 @@ import org.junit.Test;
 
 public class DirectedOutputTest {
 
-	static HashSet<Long> evenSet = new HashSet<Long>();
-	static HashSet<Long> oddSet = new HashSet<Long>();
-
-	private static class PlusTwo extends RichMapFunction<Long, Long> {
+	private static final String TEN = "ten";
+	private static final String ODD = "odd";
+	private static final String ALL = "all";
+	private static final String EVEN_AND_ODD = "evenAndOdd";
+	private static final String ODD_AND_TEN = "oddAndTen";
+	private static final String EVEN = "even";
 
+	static final class MyMap implements MapFunction<Long, Long> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public Long map(Long arg0) throws Exception {
-			arg0 += 2;
-			return arg0;
+		public Long map(Long value) throws Exception {
+			return value;
 		}
 	}
 
-	private static class EvenSink implements SinkFunction<Long> {
-
+	static final class MyOutputSelector extends OutputSelector<Long> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public void invoke(Long tuple) {
-			evenSet.add(tuple);
+		public void select(Long value, Collection<String> outputs) {
+			if (value % 2 == 0) {
+				outputs.add(EVEN);
+			} else {
+				outputs.add(ODD);
+			}
+			
+			if (value == 10L) {
+				outputs.add(TEN);
+			}
 		}
 	}
+	
+	static final class ListSink implements SinkFunction<Long> {
+		private static final long serialVersionUID = 1L;
 
-	private static class OddSink implements SinkFunction<Long> {
+		private String name;
+		private transient List<Long> list;
 
-		private static final long serialVersionUID = 1L;
+		public ListSink(String name) {
+			this.name = name;
+		}
 
 		@Override
-		public void invoke(Long tuple) {
-			oddSet.add(tuple);
+		public void invoke(Long value) {
+			list.add(value);
 		}
-	}
-
-	private static class MySelector extends OutputSelector<Long> {
-
-		private static final long serialVersionUID = 1L;
 
-		@Override
-		public void select(Long tuple, Collection<String> outputs) {
-			int mod = (int) (tuple % 2);
-			switch (mod) {
-			case 0:
-				outputs.add("ds1");
-				break;
-			case 1:
-				outputs.add("ds2");
-				break;
-			}
+		private void readObject(java.io.ObjectInputStream in) throws IOException,
+				ClassNotFoundException {
+			in.defaultReadObject();
+			outputs.put(name, new ArrayList<Long>());
+			this.list = outputs.get(name);
 		}
 	}
 
-	@SuppressWarnings("unused")
+	private static Map<String, List<Long>> outputs = new HashMap<String, List<Long>>();
+	
 	@Test
-	public void directOutputTest() throws Exception {
+	public void outputSelectorTest() throws Exception {
 
 		LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-		SplitDataStream<Long> s = env.generateSequence(1, 6).split(new MySelector(),
-				new String[] { "ds1", "ds2" });
-		DataStream<Long> ds1 = s.select("ds1").shuffle().map(new PlusTwo()).addSink(new EvenSink());
-		DataStream<Long> ds2 = s.select("ds2").map(new PlusTwo()).addSink(new OddSink());
-
-		env.executeTest(32);
-
-		HashSet<Long> expectedEven = new HashSet<Long>(Arrays.asList(4L, 6L, 8L));
-		HashSet<Long> expectedOdd = new HashSet<Long>(Arrays.asList(3L, 5L, 7L));
 
-		assertEquals(expectedEven, evenSet);
-		assertEquals(expectedOdd, oddSet);
+		SplitDataStream<Long> source = env.generateSequence(1, 10).split(new MyOutputSelector());
+		source.select(EVEN).addSink(new ListSink(EVEN));
+		source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN));
+		source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD));
+		source.selectAll().addSink(new ListSink(ALL));
+		
+		env.executeTest(128);
+		assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN));
+		assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L), outputs.get(ODD_AND_TEN));
+		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), outputs.get(EVEN_AND_ODD));
+		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), outputs.get(ALL));
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/fbfcc9eb/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 05d7494..66234d4 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
@@ -38,7 +38,7 @@ public class StreamCollectorTest {
 		sd.setInstance(new StreamRecord<Tuple1<Integer>>().setObject(new Tuple1<Integer>()));
 
 		StreamCollector<Tuple1<Integer>> collector = new StreamCollector<Tuple1<Integer>>(2, sd);
-		collector.addOutput(recWriter, new ArrayList<String>());
+		collector.addOutput(recWriter, new ArrayList<String>(), false);
 		collector.collect(new Tuple1<Integer>(3));
 		collector.collect(new Tuple1<Integer>(4));
 		collector.collect(new Tuple1<Integer>(5));


[14/18] git commit: [streaming] GroupedBatchReduce modified to batch by key, updated WindowReduce + Batch/Window reduce refactor

Posted by mb...@apache.org.
[streaming] GroupedBatchReduce modified to batch by key, updated WindowReduce + Batch/Window reduce refactor


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

Branch: refs/heads/master
Commit: 6b6951eff685669e2e444fdedf9251fb71bba423
Parents: 48d8ed7
Author: Gyula Fora <gy...@apache.org>
Authored: Tue Sep 16 17:55:08 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md                         |   4 +-
 .../api/datastream/BatchedDataStream.java       |   3 +-
 .../api/datastream/GroupedDataStream.java       | 135 +------
 .../api/datastream/WindowDataStream.java        |   2 +-
 .../api/invokable/StreamInvokable.java          |   3 +
 .../operator/BatchReduceInvokable.java          | 169 ++++++++-
 .../operator/GroupedBatchReduceInvokable.java   |  69 +---
 .../operator/GroupedWindowReduceInvokable.java  |  99 +++--
 .../operator/WindowReduceInvokable.java         | 123 ++++---
 .../streaming/state/NullableCircularBuffer.java | 362 +++++++++++++++++++
 .../api/invokable/operator/BatchReduceTest.java |   9 +-
 .../operator/GroupedBatchReduceTest.java        |  35 +-
 .../operator/WindowReduceInvokableTest.java     |  40 +-
 13 files changed, 733 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index 6ed53df..c1c6cde 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -261,8 +261,10 @@ For every incoming tuple the selected field is replaced with the current aggrega
 
 Window and batch operators allow the user to execute function on slices or windows of the DataStream in a sliding fashion. If the stepsize for the slide is not defined then the window/batchsize is used as stepsize by default.
 
-When applied to grouped data streams the operators applied will be executed on groups of elements grouped by the selected key position.
+When applied to grouped data streams the data stream will be batched/windowed for different key values separately. 
 
+For example a `ds.groupBy(0).batch(100, 10)` will produce batches of the last 100 elements for each key value with 10 record step size.
+ 
 #### Reduce on windowed/batched data streams
 The transformation calls a user-defined `ReduceFunction` on records received in the batch or during the predefined time window. The window is shifted after each reduce call. The user can also use the different streaming aggregations.
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/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 bcedac9..0249a1f 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
@@ -25,6 +25,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.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.BatchReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
@@ -199,7 +200,7 @@ public class BatchedDataStream<OUT> {
 	}
 
 	private SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
-		BatchReduceInvokable<OUT> invokable = getReduceInvokable(aggregate);
+		StreamOperatorInvokable<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/6b6951ef/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 e30d316..8978b19 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
@@ -17,16 +17,10 @@
 
 package org.apache.flink.streaming.api.datastream;
 
-import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupedReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.GroupedWindowGroupReduceInvokable;
-import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
-import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
 
@@ -73,132 +67,6 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	}
 
 	/**
-	 * Applies a group reduce transformation on preset chunks of the grouped
-	 * data stream. The {@link GroupReduceFunction} will receive input values
-	 * based on the key value. Only input values with the same key will go to
-	 * the same reducer.When the reducer has ran for all the values in the
-	 * batch, the batch is slid forward.The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
-	 * 
-	 * 
-	 * @param reducer
-	 *            The {@link GroupReduceFunction} that will be called for every
-	 *            element of the input values with the same key.
-	 * @param batchSize
-	 *            The size of the data stream chunk (the number of values in the
-	 *            batch).
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
-			int batchSize) {
-		return batchReduce(reducer, batchSize, batchSize);
-	}
-
-	/**
-	 * Applies a group reduce transformation on preset chunks of the grouped
-	 * data stream in a sliding window fashion. The {@link GroupReduceFunction}
-	 * will receive input values based on the key value. Only input values with
-	 * the same key will go to the same reducer. When the reducer has ran for
-	 * all the values in the batch, the batch is slid forward. The user can also
-	 * extend {@link RichGroupReduceFunction} to gain access to other features
-	 * provided by the {@link RichFuntion} interface.
-	 * 
-	 * @param reducer
-	 *            The {@link GroupReduceFunction} that will be called for every
-	 *            element of the input values with the same key.
-	 * @param batchSize
-	 *            The size of the data stream chunk (the number of values in the
-	 *            batch).
-	 * @param slideSize
-	 *            The number of values the batch is slid by.
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> batchReduce(GroupReduceFunction<OUT, R> reducer,
-			long batchSize, long slideSize) {
-
-		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
-				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1), new GroupedBatchGroupReduceInvokable<OUT, R>(reducer,
-				batchSize, slideSize, keyPosition));
-	}
-
-	/**
-	 * Applies a group reduce transformation on preset "time" chunks of the
-	 * grouped data stream. The {@link GroupReduceFunction} will receive input
-	 * values based on the key value. Only input values with the same key will
-	 * go to the same reducer.When the reducer has ran for all the values in the
-	 * batch, the window is shifted forward. The user can also extend
-	 * {@link RichGroupReduceFunction} to gain access to other features provided
-	 * by the {@link RichFuntion} interface.
-	 * 
-	 * 
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
-	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize) {
-		return windowReduce(reducer, windowSize, windowSize);
-	}
-
-	/**
-	 * Applies a group reduce transformation on preset "time" chunks of the
-	 * grouped data stream in a sliding window fashion. The
-	 * {@link GroupReduceFunction} will receive input values based on the key
-	 * value. Only input values with the same key will go to the same reducer.
-	 * When the reducer has ran for all the values in the batch, the window is
-	 * shifted forward. The user can also extend {@link RichGroupReduceFunction}
-	 * to gain access to other features provided by the {@link RichFuntion}
-	 * interface.
-	 *
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
-	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
-	 * @param slideInterval
-	 *            The time interval the batch is slid by.
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize, long slideInterval) {
-		return windowReduce(reducer, windowSize, slideInterval, new DefaultTimeStamp<OUT>());
-	}
-
-	/**
-	 * Applies a group reduce transformation on preset "time" chunks of the
-	 * grouped data stream in a sliding window fashion. The
-	 * {@link GroupReduceFunction} will receive input values based on the key
-	 * value. Only input values with the same key will go to the same reducer.
-	 * When the reducer has ran for all the values in the batch, the window is
-	 * shifted forward. The time is determined by a user-defined timestamp. The
-	 * user can also extend {@link RichGroupReduceFunction} to gain access to
-	 * other features provided by the {@link RichFuntion} interface.
-	 *
-	 * @param reducer
-	 *            The GroupReduceFunction that is called for each time window.
-	 * @param windowSize
-	 *            SingleOutputStreamOperator The time window to run the reducer
-	 *            on, in milliseconds.
-	 * @param slideInterval
-	 *            The time interval the batch is slid by.
-	 * @param timestamp
-	 *            Timestamp function to retrieve a timestamp from an element.
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> windowReduce(GroupReduceFunction<OUT, R> reducer,
-			long windowSize, long slideInterval, TimeStamp<OUT> timestamp) {
-		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
-				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1), new GroupedWindowGroupReduceInvokable<OUT, R>(reducer,
-				windowSize, slideInterval, keyPosition, timestamp));
-	}
-
-	/**
 	 * Applies an aggregation that sums the grouped data stream at the given
 	 * position, grouped by the given key position. Input values with the same
 	 * key will be summed.
@@ -240,7 +108,8 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 	@Override
 	protected SingleOutputStreamOperator<OUT, ?> aggregate(AggregationFunction<OUT> aggregate) {
 
-		GroupedReduceInvokable<OUT> invokable = new GroupedReduceInvokable<OUT>(aggregate, keyPosition);
+		GroupedReduceInvokable<OUT> invokable = new GroupedReduceInvokable<OUT>(aggregate,
+				keyPosition);
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("groupReduce", aggregate,
 				outTypeWrapper, outTypeWrapper, invokable);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
index 4756050..7a53d1a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowDataStream.java
@@ -57,7 +57,7 @@ public class WindowDataStream<OUT> extends BatchedDataStream<OUT> {
 	}
 
 	protected BatchReduceInvokable<OUT> getReduceInvokable(ReduceFunction<OUT> reducer) {
-		BatchReduceInvokable<OUT> invokable;
+		WindowReduceInvokable<OUT> invokable;
 		if (isGrouped) {
 			invokable = new GroupedWindowReduceInvokable<OUT>(reducer, batchSize, slideSize,
 					keyPosition, timeStamp);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/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 9a6f2cc..342cc7a 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
@@ -37,6 +37,7 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 
 	protected Collector<OUT> collector;
 	protected Function userFunction;
+	protected volatile boolean isRunning;
 
 	public StreamInvokable(Function userFunction) {
 		this.userFunction = userFunction;
@@ -54,6 +55,7 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 	 *            The configuration parameters for the operator
 	 */
 	public void open(Configuration parameters) throws Exception {
+		isRunning=true;
 		if (userFunction instanceof RichFunction) {
 			((RichFunction) userFunction).open(parameters);
 		}
@@ -65,6 +67,7 @@ public abstract class StreamInvokable<OUT> implements Serializable {
 	 * 
 	 */
 	public void close() throws Exception {
+		isRunning = false;
 		if (userFunction instanceof RichFunction) {
 			((RichFunction) userFunction).close();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/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 cfca1ab..d713f6d 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
@@ -17,64 +17,193 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
+import java.io.Serializable;
 import java.util.Iterator;
 
+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.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.NullableCircularBuffer;
 
-public class BatchReduceInvokable<OUT> extends BatchGroupReduceInvokable<OUT, OUT> {
+public class BatchReduceInvokable<OUT> extends StreamOperatorInvokable<OUT, OUT> {
 
 	private static final long serialVersionUID = 1L;
 	protected ReduceFunction<OUT> reducer;
 	protected TypeSerializer<OUT> typeSerializer;
-	protected OUT reduceReuse;
+
+	protected long slideSize;
+
+	protected long batchSize;
+	protected int granularity;
+	protected long batchPerSlide;
+	protected long numberOfBatches;
+	protected StreamBatch batch;
+	protected StreamBatch currentBatch;
 
 	public BatchReduceInvokable(ReduceFunction<OUT> reduceFunction, long batchSize, long slideSize) {
-		super(null, batchSize, slideSize);
+		super(reduceFunction);
 		this.reducer = reduceFunction;
+		this.batchSize = batchSize;
+		this.slideSize = slideSize;
+		this.granularity = (int) MathUtils.gcd(batchSize, slideSize);
+		this.batchPerSlide = slideSize / granularity;
+		this.numberOfBatches = batchSize / granularity;
+		this.batch = new StreamBatch();
 	}
 
-	protected void collectOneUnit() throws Exception {
-		OUT reduced = null;
-		if (batchNotFull()) {
-			reduced = reuse.getObject();
+	@Override
+	protected void immutableInvoke() throws Exception {
+		if ((reuse = recordIterator.next(reuse)) == null) {
+			throw new RuntimeException("DataStream must not be empty");
+
+		}
+
+		while (reuse != null) {		
+			StreamBatch batch = getBatch(reuse);
+
+			batch.reduceToBuffer(reuse);
+
 			resetReuse();
-			while (getNextRecord() != null && batchNotFull()) {
-				reduced = reducer.reduce(reduced, reuse.getObject());
-				resetReuse();
-			}
+			reuse = recordIterator.next(reuse);
 		}
-		state.pushBack(reduced);
+		
+		reduceLastBatch();
+
+	}
+
+	protected void reduceLastBatch() throws Exception {
+		batch.reduceLastBatch();		
+	}
+
+	protected StreamBatch getBatch(StreamRecord<OUT> next) {
+		return batch;
 	}
 
 	@Override
-	protected void reduce() {
+	// TODO: implement mutableInvoke for reduce
+	protected void mutableInvoke() throws Exception {
+		System.out.println("Immutable setting is used");
+		immutableInvoke();
+	}
+
+	protected void reduce(StreamBatch batch) {
+		this.currentBatch = batch;
 		callUserFunctionAndLogException();
 	}
 
 	@Override
 	protected void callUserFunction() throws Exception {
-		Iterator<OUT> reducedIterator = state.getBufferIterator();
-		OUT reduced;
-		do {
+		Iterator<OUT> reducedIterator = currentBatch.getIterator();
+		OUT reduced = null;
+
+		while (reducedIterator.hasNext() && reduced == null) {
 			reduced = reducedIterator.next();
-		} while (reducedIterator.hasNext() && reduced == null);
+		}
 
 		while (reducedIterator.hasNext()) {
 			OUT next = reducedIterator.next();
 			if (next != null) {
-				next = typeSerializer.copy(next, reduceReuse);
 				reduced = reducer.reduce(reduced, next);
 			}
 		}
-		collector.collect(reduced);
+		if (reduced != null) {
+			collector.collect(reduced);
+		}
 	}
 
 	@Override
 	public void open(Configuration config) throws Exception {
 		super.open(config);
 		this.typeSerializer = serializer.getObjectSerializer();
-		this.reduceReuse = typeSerializer.createInstance();
 	}
+
+	protected class StreamBatch implements Serializable {
+
+		private static final long serialVersionUID = 1L;
+		protected long counter;
+		protected long minibatchCounter;
+		protected OUT currentValue;
+
+		protected NullableCircularBuffer circularBuffer;
+
+		public StreamBatch() {
+
+			this.circularBuffer = new NullableCircularBuffer((int) (batchSize / granularity));
+			this.counter = 0;
+			this.minibatchCounter = 0;
+
+		}
+
+		public void reduceToBuffer(StreamRecord<OUT> next) throws Exception {
+			OUT nextValue = next.getObject();
+			if (currentValue != null) {
+				currentValue = reducer.reduce(currentValue, nextValue);
+			} else {
+				currentValue = nextValue;
+			}
+
+			counter++;
+
+			if (miniBatchEnd()) {
+				addToBuffer();
+				if (batchEnd()) {
+					reduceBatch();
+				}
+			}
+
+		}
+
+		protected void addToBuffer() {
+			circularBuffer.add(currentValue);
+			minibatchCounter++;
+			currentValue = null;
+		}
+
+		protected boolean miniBatchEnd() {
+			return (counter % granularity) == 0;
+		}
+
+		public boolean batchEnd() {
+			if (counter == batchSize) {
+				counter -= slideSize;
+				minibatchCounter -= batchPerSlide;
+				return true;
+			}
+			return false;
+		}
+
+		public void reduceLastBatch() throws Exception {
+			if (miniBatchInProgress()) {
+				addToBuffer();
+			}
+
+			if (minibatchCounter >= 0) {
+				for (long i = 0; i < (numberOfBatches - minibatchCounter); i++) {
+					circularBuffer.remove();
+				}
+				if (!circularBuffer.isEmpty()) {
+					reduce(this);
+				}
+			}
+
+		}
+		
+		public boolean miniBatchInProgress(){
+			return currentValue != null;
+		}
+
+		public void reduceBatch() {
+			reduce(this);
+		}
+
+		@SuppressWarnings("unchecked")
+		public Iterator<OUT> getIterator() {
+			return circularBuffer.iterator();
+		}
+
+	}
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
index c173932..f842073 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
@@ -18,77 +18,40 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.state.SlidingWindowState;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
 public class GroupedBatchReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 
 	private static final long serialVersionUID = 1L;
-	protected transient SlidingWindowState<Map<Object, OUT>> intermediateValues;
-
-	private int keyPosition;
+	int keyPosition;
+	Map<Object, StreamBatch> streamBatches;
 
 	public GroupedBatchReduceInvokable(ReduceFunction<OUT> reduceFunction, long batchSize,
 			long slideSize, int keyPosition) {
 		super(reduceFunction, batchSize, slideSize);
 		this.keyPosition = keyPosition;
-	}
-
-	protected void collectOneUnit() throws Exception {
-		Map<Object, OUT> values = new HashMap<Object, OUT>();
-		if (batchNotFull()) {
-			do {
-				Object key = reuse.getField(keyPosition);
-				OUT nextValue = reuse.getObject();
-				OUT currentValue = values.get(key);
-				if (currentValue == null) {
-					values.put(key, nextValue);
-				} else {
-					values.put(key, reducer.reduce(currentValue, nextValue));
-				}
-				resetReuse();
-			} while (getNextRecord() != null && batchNotFull());
-		}
-		intermediateValues.pushBack(values);
+		this.streamBatches = new HashMap<Object, StreamBatch>();
 	}
 
 	@Override
-	protected boolean isStateFull() {
-		return intermediateValues.isFull();
+	protected void reduceLastBatch() throws Exception {
+		for(StreamBatch batch: streamBatches.values()){
+			batch.reduceLastBatch();
+		}		
 	}
 
 	@Override
-	protected void callUserFunction() throws Exception {
-		Iterator<Map<Object, OUT>> reducedIterator = intermediateValues.getBufferIterator();
-		Map<Object, OUT> reducedValues = reducedIterator.next();
-
-		while (reducedIterator.hasNext()) {
-			Map<Object, OUT> nextValues = reducedIterator.next();
-			for (Entry<Object, OUT> entry : nextValues.entrySet()) {
-				OUT currentValue = reducedValues.get(entry.getKey());
-				if (currentValue == null) {
-					reducedValues.put(entry.getKey(), entry.getValue());
-				} else {
-					OUT next = typeSerializer.copy(entry.getValue(), reduceReuse);
-					reducedValues.put(entry.getKey(), reducer.reduce(currentValue, next));
-				}
-			}
+	protected StreamBatch getBatch(StreamRecord<OUT> next) {
+		Object key = next.getField(keyPosition);
+		StreamBatch batch = streamBatches.get(key);
+		if(batch == null){
+			batch=new StreamBatch();
+			streamBatches.put(key, batch);
 		}
-		for (OUT value : reducedValues.values()) {
-			collector.collect(value);
-		}
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.intermediateValues = new SlidingWindowState<Map<Object, OUT>>(batchSize, slideSize,
-				granularity);
+		return batch;
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
index df94843..1214347 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
@@ -17,44 +17,99 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
-import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
-public class GroupedWindowReduceInvokable<OUT> extends GroupedBatchReduceInvokable<OUT> {
+public class GroupedWindowReduceInvokable<OUT> extends WindowReduceInvokable<OUT> {
 
 	private static final long serialVersionUID = 1L;
-
-	private TimeStamp<OUT> timestamp;
-	private long startTime;
-	private long nextRecordTime;
+	private int keyPosition;
 
 	public GroupedWindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
 			long slideInterval, int keyPosition, TimeStamp<OUT> timestamp) {
-		super(reduceFunction, windowSize, slideInterval, keyPosition);
-		this.timestamp = timestamp;
-		this.startTime = timestamp.getStartTime();
+		super(reduceFunction, windowSize, slideInterval, timestamp);
+		this.keyPosition = keyPosition;
+		this.window = new GroupedStreamWindow();
+		this.batch = this.window;
 	}
-
+	
 	@Override
-	protected StreamRecord<OUT> getNextRecord() throws IOException {
-		reuse = recordIterator.next(reuse);
-		if (reuse != null) {
-			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
+	protected void callUserFunction() throws Exception {	
+		@SuppressWarnings("unchecked")
+		Iterator<Map<Object, OUT>> reducedIterator = (Iterator<Map<Object, OUT>>) batch.getIterator();
+		Map<Object, OUT> reducedValues = reducedIterator.next();
+
+		while (reducedIterator.hasNext()) {
+			Map<Object, OUT> nextValues = reducedIterator.next();
+			for (Entry<Object, OUT> entry : nextValues.entrySet()) {
+				OUT currentValue = reducedValues.get(entry.getKey());
+				if (currentValue == null) {
+					reducedValues.put(entry.getKey(), entry.getValue());
+				} else {
+					reducedValues.put(entry.getKey(), reducer.reduce(currentValue, entry.getValue()));
+				}
+			}
+		}
+		for (OUT value : reducedValues.values()) {
+			collector.collect(value);
 		}
-		return reuse;
 	}
+	
 
-	@Override
-	protected boolean batchNotFull() {
-		if (nextRecordTime < startTime + granularity) {
-			return true;
-		} else {
-			startTime += granularity;
-			return false;
+	protected class GroupedStreamWindow extends StreamWindow {
+
+		private static final long serialVersionUID = 1L;
+		private Map<Object, OUT> currentValues;
+
+		public GroupedStreamWindow() {
+			super();
+			this.currentValues  = new HashMap<Object, OUT>();
 		}
+
+		@Override
+		public void reduceToBuffer(StreamRecord<OUT> next) throws Exception {
+
+			OUT nextValue = next.getObject();
+			Object key = next.getField(keyPosition);
+			checkBatchEnd(timestamp.getTimestamp(nextValue));
+
+			OUT currentValue = currentValues.get(key);
+			if (currentValue != null) {
+				currentValues.put(key, reducer.reduce(currentValue, nextValue));
+			}else{
+				currentValues.put(key, nextValue);
+			}
+
+		}
+		
+		@Override
+		public boolean miniBatchInProgress() {
+			return !currentValues.isEmpty();
+		};
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected void addToBuffer() {
+			Map<Object, OUT> reuseMap;
+			
+			if (circularBuffer.isFull()) {
+				reuseMap = (Map<Object, OUT>) circularBuffer.remove();
+				reuseMap.clear();
+			} else {
+				reuseMap = new HashMap<Object, OUT>(currentValues.size());
+			}
+			
+			circularBuffer.add(currentValues);
+			minibatchCounter++;
+			currentValues = reuseMap;
+		}
+
 	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index bd51c65..b653f95 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -17,88 +17,105 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
-import java.io.IOException;
-import java.util.Iterator;
-
 import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
 import org.apache.flink.streaming.api.invokable.util.TimeStamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
 public class WindowReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 	private static final long serialVersionUID = 1L;
-	private long startTime;
-	private long nextRecordTime;
-	private TimeStamp<OUT> timestamp;
-	private String nullElement = "nullElement";
+	protected long startTime;
+	protected long nextRecordTime;
+	protected TimeStamp<OUT> timestamp;
+	protected StreamWindow window;
 
 	public WindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
 			long slideInterval, TimeStamp<OUT> timestamp) {
 		super(reduceFunction, windowSize, slideInterval);
 		this.timestamp = timestamp;
 		this.startTime = timestamp.getStartTime();
+		this.window = new StreamWindow();
+		this.batch = this.window;
 	}
 
-	protected StreamRecord<OUT> getNextRecord() throws IOException {
-		reuse = recordIterator.next(reuse);
-		if (reuse != null) {
-			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
+	protected class StreamWindow extends StreamBatch {
+
+		private static final long serialVersionUID = 1L;
+
+		public StreamWindow() {
+			super();
+
 		}
-		return reuse;
-	}
 
-	@Override
-	protected boolean batchNotFull() {
-		if (nextRecordTime < startTime + granularity) {
-			return true;
-		} else {
-			startTime += granularity;
-			return false;
+		@Override
+		public void reduceToBuffer(StreamRecord<OUT> next) throws Exception {
+			OUT nextValue = next.getObject();
+			
+			checkBatchEnd(timestamp.getTimestamp(nextValue));
+			
+			if (currentValue != null) {
+				currentValue = reducer.reduce(currentValue, nextValue);
+			} else {
+				currentValue = nextValue;
+			}
 		}
-	}
 
-	@Override
-	protected void collectOneUnit() throws Exception {
-		OUT reduced = null;
-		if (batchNotFull()) {
-			reduced = reuse.getObject();
-			resetReuse();
-			while (getNextRecord() != null && batchNotFull()) {
-				reduced = reducer.reduce(reduced, reuse.getObject());
-				resetReuse();
+		protected synchronized void checkBatchEnd(long timeStamp) {
+			nextRecordTime = timeStamp;
+
+			while (miniBatchEnd()) {
+				addToBuffer();
+				if (batchEnd()) {
+					reduceBatch();
+				}
 			}
 		}
-		if (reduced != null) {
-			state.pushBack(reduced);
-		} else {
-			state.pushBack(nullElement);
+
+		@Override
+		protected boolean miniBatchEnd() {
+			if (nextRecordTime < startTime + granularity) {
+				return false;
+			} else {
+				startTime += granularity;
+				return true;
+			}
+		}
+
+		@Override
+		public boolean batchEnd() {
+			if (minibatchCounter == numberOfBatches) {
+				minibatchCounter -= batchPerSlide;
+				return true;
+			}
+			return false;
 		}
+
 	}
 
 	@Override
-	protected void callUserFunction() throws Exception {
-		Iterator<OUT> reducedIterator = state.getBufferIterator();
-		OUT reduced = null;
-		do {
-			OUT next = reducedIterator.next();
-			if (next != nullElement) {
-				reduced = next;
-			}
-		} while (reducedIterator.hasNext() && reduced == null);
+	public void open(Configuration config) throws Exception {
+		super.open(config);
+		if (timestamp instanceof DefaultTimeStamp) {
+			(new TimeCheck()).start();
+		}
+	}
 
-		while (reducedIterator.hasNext()) {
-			OUT next = reducedIterator.next();
-			if (next != null) {
+	private class TimeCheck extends Thread {
+		@Override
+		public void run() {
+			while (true) {
 				try {
-					next = typeSerializer.copy(next, reduceReuse);
-					reduced = reducer.reduce(reduced, next);
-				} catch (ClassCastException e) {
-					// nullElement in buffer
+					Thread.sleep(slideSize);
+				} catch (InterruptedException e) {
+				}
+				if (isRunning) {
+					window.checkBatchEnd(System.currentTimeMillis());
+				} else {
+					break;
 				}
 			}
 		}
-		if (reduced != null) {
-			collector.collect(reduced);
-		}
 	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/NullableCircularBuffer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/NullableCircularBuffer.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/NullableCircularBuffer.java
new file mode 100644
index 0000000..d8e0116
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/NullableCircularBuffer.java
@@ -0,0 +1,362 @@
+/**
+ * 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.state;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.AbstractCollection;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.commons.collections.BoundedCollection;
+import org.apache.commons.collections.Buffer;
+import org.apache.commons.collections.BufferUnderflowException;
+
+@SuppressWarnings("rawtypes")
+public class NullableCircularBuffer extends AbstractCollection implements Buffer,
+		BoundedCollection, Serializable {
+
+	/** Serialization version */
+	private static final long serialVersionUID = 5603722811189451017L;
+
+	/** Underlying storage array */
+	private transient Object[] elements;
+
+	/** Array index of first (oldest) buffer element */
+	private transient int start = 0;
+
+	/**
+	 * Index mod maxElements of the array position following the last buffer
+	 * element. Buffer elements start at elements[start] and "wrap around"
+	 * elements[maxElements-1], ending at elements[decrement(end)]. For example,
+	 * elements = {c,a,b}, start=1, end=1 corresponds to the buffer [a,b,c].
+	 */
+	private transient int end = 0;
+
+	/** Flag to indicate if the buffer is currently full. */
+	private transient boolean full = false;
+
+	/** Capacity of the buffer */
+	private final int maxElements;
+
+	/**
+	 * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold 32
+	 * elements.
+	 */
+	public NullableCircularBuffer() {
+		this(32);
+	}
+
+	/**
+	 * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold the
+	 * specified number of elements.
+	 *
+	 * @param size
+	 *            the maximum number of elements for this fifo
+	 * @throws IllegalArgumentException
+	 *             if the size is less than 1
+	 */
+	public NullableCircularBuffer(int size) {
+		if (size <= 0) {
+			throw new IllegalArgumentException("The size must be greater than 0");
+		}
+		elements = new Object[size];
+		maxElements = elements.length;
+	}
+
+	/**
+	 * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold all of
+	 * the elements in the specified collection. That collection's elements will
+	 * also be added to the buffer.
+	 *
+	 * @param coll
+	 *            the collection whose elements to add, may not be null
+	 * @throws NullPointerException
+	 *             if the collection is null
+	 */
+	@SuppressWarnings("unchecked")
+	public NullableCircularBuffer(Collection coll) {
+		this(coll.size());
+		addAll(coll);
+	}
+
+	// -----------------------------------------------------------------------
+	/**
+	 * Write the buffer out using a custom routine.
+	 * 
+	 * @param out
+	 *            the output stream
+	 * @throws IOException
+	 */
+	private void writeObject(ObjectOutputStream out) throws IOException {
+		out.defaultWriteObject();
+		out.writeInt(size());
+		for (Iterator it = iterator(); it.hasNext();) {
+			out.writeObject(it.next());
+		}
+	}
+
+	/**
+	 * Read the buffer in using a custom routine.
+	 * 
+	 * @param in
+	 *            the input stream
+	 * @throws IOException
+	 * @throws ClassNotFoundException
+	 */
+	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
+		elements = new Object[maxElements];
+		int size = in.readInt();
+		for (int i = 0; i < size; i++) {
+			elements[i] = in.readObject();
+		}
+		start = 0;
+		full = (size == maxElements);
+		if (full) {
+			end = 0;
+		} else {
+			end = size;
+		}
+	}
+
+	// -----------------------------------------------------------------------
+	/**
+	 * Returns the number of elements stored in the buffer.
+	 *
+	 * @return this buffer's size
+	 */
+	public int size() {
+		int size = 0;
+
+		if (end < start) {
+			size = maxElements - start + end;
+		} else if (end == start) {
+			size = (full ? maxElements : 0);
+		} else {
+			size = end - start;
+		}
+
+		return size;
+	}
+
+	/**
+	 * Returns true if this buffer is empty; false otherwise.
+	 *
+	 * @return true if this buffer is empty
+	 */
+	public boolean isEmpty() {
+		return size() == 0;
+	}
+
+	/**
+	 * Returns true if this collection is full and no new elements can be added.
+	 *
+	 * @return <code>true</code> if the collection is full
+	 */
+	public boolean isFull() {
+		return size() == maxElements;
+	}
+
+	/**
+	 * Gets the maximum size of the collection (the bound).
+	 *
+	 * @return the maximum number of elements the collection can hold
+	 */
+	public int maxSize() {
+		return maxElements;
+	}
+
+	/**
+	 * Clears this buffer.
+	 */
+	public void clear() {
+		full = false;
+		start = 0;
+		end = 0;
+		Arrays.fill(elements, null);
+	}
+
+	/**
+	 * Adds the given element to this buffer.
+	 *
+	 * @param element
+	 *            the element to add
+	 * @return true, always
+	 */
+	public boolean add(Object element) {
+
+		if (isFull()) {
+			remove();
+		}
+
+		elements[end++] = element;
+
+		if (end >= maxElements) {
+			end = 0;
+		}
+
+		if (end == start) {
+			full = true;
+		}
+
+		return true;
+	}
+
+	/**
+	 * Returns the least recently inserted element in this buffer.
+	 *
+	 * @return the least recently inserted element
+	 * @throws BufferUnderflowException
+	 *             if the buffer is empty
+	 */
+	public Object get() {
+		if (isEmpty()) {
+			throw new BufferUnderflowException("The buffer is already empty");
+		}
+
+		return elements[start];
+	}
+
+	/**
+	 * Removes the least recently inserted element from this buffer.
+	 *
+	 * @return the least recently inserted element
+	 * @throws BufferUnderflowException
+	 *             if the buffer is empty
+	 */
+	public Object remove() {
+		if (isEmpty()) {
+			throw new BufferUnderflowException("The buffer is already empty");
+		}
+
+		Object element = elements[start];
+
+		elements[start++] = null;
+
+		if (start >= maxElements) {
+			start = 0;
+		}
+
+		full = false;
+
+		return element;
+	}
+
+	/**
+	 * Increments the internal index.
+	 * 
+	 * @param index
+	 *            the index to increment
+	 * @return the updated index
+	 */
+	private int increment(int index) {
+		index++;
+		if (index >= maxElements) {
+			index = 0;
+		}
+		return index;
+	}
+
+	/**
+	 * Decrements the internal index.
+	 * 
+	 * @param index
+	 *            the index to decrement
+	 * @return the updated index
+	 */
+	private int decrement(int index) {
+		index--;
+		if (index < 0) {
+			index = maxElements - 1;
+		}
+		return index;
+	}
+
+	/**
+	 * Returns an iterator over this buffer's elements.
+	 *
+	 * @return an iterator over this buffer's elements
+	 */
+	public Iterator iterator() {
+		return new Iterator() {
+
+			private int index = start;
+			private int lastReturnedIndex = -1;
+			private boolean isFirst = full;
+
+			public boolean hasNext() {
+				return isFirst || (index != end);
+
+			}
+
+			public Object next() {
+				if (!hasNext()) {
+					throw new NoSuchElementException();
+				}
+				isFirst = false;
+				lastReturnedIndex = index;
+				index = increment(index);
+				return elements[lastReturnedIndex];
+			}
+
+			public void remove() {
+				if (lastReturnedIndex == -1) {
+					throw new IllegalStateException();
+				}
+
+				// First element can be removed quickly
+				if (lastReturnedIndex == start) {
+					NullableCircularBuffer.this.remove();
+					lastReturnedIndex = -1;
+					return;
+				}
+
+				int pos = lastReturnedIndex + 1;
+				if (start < lastReturnedIndex && pos < end) {
+					// shift in one part
+					System.arraycopy(elements, pos, elements, lastReturnedIndex, end - pos);
+				} else {
+					// Other elements require us to shift the subsequent
+					// elements
+					while (pos != end) {
+						if (pos >= maxElements) {
+							elements[pos - 1] = elements[0];
+							pos = 0;
+						} else {
+							elements[decrement(pos)] = elements[pos];
+							pos = increment(pos);
+						}
+					}
+				}
+
+				lastReturnedIndex = -1;
+				end = decrement(end);
+				elements[end] = null;
+				full = false;
+				index = decrement(index);
+			}
+
+		};
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
index 3140dc0..f70412f 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -50,7 +50,7 @@ public class BatchReduceTest {
 		expected.add(12);
 		expected.add(18);
 		expected.add(24);
-		expected.add(27);
+		expected.add(19);
 		assertEquals(expected, MockInvokable.createAndExecute(invokable, inputs));
 
 		List<Integer> inputs2 = new ArrayList<Integer>();
@@ -58,7 +58,7 @@ public class BatchReduceTest {
 		inputs2.add(2);
 		inputs2.add(-1);
 		inputs2.add(-3);
-		inputs2.add(3);
+		inputs2.add(-4);
 
 		BatchReduceInvokable<Integer> invokable2 = new BatchReduceInvokable<Integer>(
 				new ReduceFunction<Integer>() {
@@ -72,12 +72,11 @@ public class BatchReduceTest {
 							return value2;
 						}
 					}
-				}, 2, 2);
+				}, 2, 3);
 
 		List<Integer> expected2 = new ArrayList<Integer>();
 		expected2.add(1);
-		expected2.add(-3);
-		expected2.add(3);
+		expected2.add(-4);
 
 		assertEquals(expected2, MockInvokable.createAndExecute(invokable2, inputs2));
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
index 850ac8d..00550f0 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
@@ -44,6 +44,15 @@ public class GroupedBatchReduceTest {
 		inputs.add(5);
 		inputs.add(1);
 		inputs.add(5);
+		
+		List<Integer> expected = new ArrayList<Integer>();
+		expected.add(15);
+		expected.add(3);
+		expected.add(3);
+		expected.add(15);
+		expected.add(1);
+		expected.add(5);
+
 
 		GroupedBatchReduceInvokable<Integer> invokable = new GroupedBatchReduceInvokable<Integer>(
 				new ReduceFunction<Integer>() {
@@ -53,17 +62,8 @@ public class GroupedBatchReduceTest {
 					public Integer reduce(Integer value1, Integer value2) throws Exception {
 						return value1 + value2;
 					}
-				}, 4, 2, 0);
-
-		List<Integer> expected = new ArrayList<Integer>();
-		expected.add(2);
-		expected.add(10);
-		expected.add(1);
-		expected.add(15);
-		expected.add(2);
-		expected.add(10);
-		expected.add(2);
-		expected.add(10);
+				}, 3, 2, 0);
+		
 		List<Integer> actual = MockInvokable.createAndExecute(invokable, inputs);
 		assertEquals(new HashSet<Integer>(expected), new HashSet<Integer>(actual));
 		assertEquals(expected.size(), actual.size());
@@ -77,6 +77,11 @@ public class GroupedBatchReduceTest {
 		inputs2.add(new Tuple2<Integer, String>(10, "a"));
 		inputs2.add(new Tuple2<Integer, String>(2, "b"));
 		inputs2.add(new Tuple2<Integer, String>(1, "a"));
+		
+		List<Tuple2<Integer, String>> expected2 = new ArrayList<Tuple2<Integer, String>>();
+		expected2.add(new Tuple2<Integer, String>(-1, "a"));
+		expected2.add(new Tuple2<Integer, String>(-2, "a"));
+		expected2.add(new Tuple2<Integer, String>(0, "b"));
 
 		GroupedBatchReduceInvokable<Tuple2<Integer, String>> invokable2 = new GroupedBatchReduceInvokable<Tuple2<Integer, String>>(
 				new ReduceFunction<Tuple2<Integer, String>>() {
@@ -93,14 +98,10 @@ public class GroupedBatchReduceTest {
 					}
 				}, 3, 3, 1);
 
-		List<Tuple2<Integer, String>> expected2 = new ArrayList<Tuple2<Integer, String>>();
-		expected2.add(new Tuple2<Integer, String>(1, "a"));
-		expected2.add(new Tuple2<Integer, String>(0, "b"));
-		expected2.add(new Tuple2<Integer, String>(-2, "a"));
-		expected2.add(new Tuple2<Integer, String>(2, "b"));
-		expected2.add(new Tuple2<Integer, String>(1, "a"));
+		
 
 		List<Tuple2<Integer, String>> actual2 = MockInvokable.createAndExecute(invokable2, inputs2);
+		
 		assertEquals(new HashSet<Tuple2<Integer, String>>(expected2),
 				new HashSet<Tuple2<Integer, String>>(actual2));
 		assertEquals(expected2.size(), actual2.size());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6b6951ef/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
index ff0951d..8b4431a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
@@ -44,6 +44,17 @@ public class WindowReduceInvokableTest {
 		inputs.add(10);
 		inputs.add(11);
 		inputs.add(11);
+		//1,2,3,4-3,4,5,6-5,6,7,8-7,8,9,10-9,10,11
+		//12-12-5-10-32
+		
+		List<Integer> expected = new ArrayList<Integer>();
+		expected.add(12);
+		expected.add(12);
+		expected.add(5);
+		expected.add(10);
+		expected.add(32);
+
+		
 		WindowReduceInvokable<Integer> invokable = new WindowReduceInvokable<Integer>(
 				new ReduceFunction<Integer>() {
 					private static final long serialVersionUID = 1L;
@@ -62,15 +73,11 @@ public class WindowReduceInvokableTest {
 
 					@Override
 					public long getStartTime() {
-						return 0;
+						return 1;
 					}
 				});
 
-		List<Integer> expected = new ArrayList<Integer>();
-		expected.add(8);
-		expected.add(16);
-		expected.add(9);
-		expected.add(32);
+		
 		assertEquals(expected, MockInvokable.createAndExecute(invokable, inputs));
 
 		List<Tuple2<String, Integer>> inputs2 = new ArrayList<Tuple2<String, Integer>>();
@@ -78,9 +85,18 @@ public class WindowReduceInvokableTest {
 		inputs2.add(new Tuple2<String, Integer>("a", 2));
 		inputs2.add(new Tuple2<String, Integer>("b", 2));
 		inputs2.add(new Tuple2<String, Integer>("b", 2));
-		inputs2.add(new Tuple2<String, Integer>("a", 3));
-		inputs2.add(new Tuple2<String, Integer>("b", 4));
 		inputs2.add(new Tuple2<String, Integer>("b", 5));
+		inputs2.add(new Tuple2<String, Integer>("a", 7));
+		inputs2.add(new Tuple2<String, Integer>("b", 9));
+		inputs2.add(new Tuple2<String, Integer>("b", 10));
+		
+		List<Tuple2<String, Integer>> expected2 = new ArrayList<Tuple2<String, Integer>>();
+		expected2.add(new Tuple2<String, Integer>("a", 3));
+		expected2.add(new Tuple2<String, Integer>("b", 4));
+		expected2.add(new Tuple2<String, Integer>("b", 5));
+		expected2.add(new Tuple2<String, Integer>("a", 7));
+		expected2.add(new Tuple2<String, Integer>("b", 10));
+
 
 		GroupedWindowReduceInvokable<Tuple2<String, Integer>> invokable2 = new GroupedWindowReduceInvokable<Tuple2<String, Integer>>(
 				new ReduceFunction<Tuple2<String, Integer>>() {
@@ -91,7 +107,7 @@ public class WindowReduceInvokableTest {
 							Tuple2<String, Integer> value2) throws Exception {
 						return new Tuple2<String, Integer>(value1.f0, value1.f1 + value2.f1);
 					}
-				}, 3, 2, 0, new TimeStamp<Tuple2<String, Integer>>() {
+				}, 2, 3, 0, new TimeStamp<Tuple2<String, Integer>>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
@@ -105,11 +121,7 @@ public class WindowReduceInvokableTest {
 					}
 				});
 
-		List<Tuple2<String, Integer>> expected2 = new ArrayList<Tuple2<String, Integer>>();
-		expected2.add(new Tuple2<String, Integer>("a", 6));
-		expected2.add(new Tuple2<String, Integer>("b", 4));
-		expected2.add(new Tuple2<String, Integer>("b", 9));
-		expected2.add(new Tuple2<String, Integer>("a", 3));
+
 		List<Tuple2<String, Integer>> actual2 = MockInvokable.createAndExecute(invokable2, inputs2);
 		assertEquals(new HashSet<Tuple2<String, Integer>>(expected2),
 				new HashSet<Tuple2<String, Integer>>(actual2));


[05/18] git commit: [FLINK-1080] [streaming] Streaming aggregation added

Posted by mb...@apache.org.
[FLINK-1080] [streaming] Streaming aggregation added

Conflicts:
	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/0c8f1dac
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/0c8f1dac
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/0c8f1dac

Branch: refs/heads/master
Commit: 0c8f1dacc2d97b9b4786ac4f0ccf149ed6998d46
Parents: fbfcc9e
Author: Eszes Dávid <es...@gmail.com>
Authored: Tue Sep 2 15:58:14 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../connectors/twitter/TwitterLocal.java        |   3 +-
 .../flink/streaming/api/JobGraphBuilder.java    |  34 ++++++
 .../api/collector/StreamCollector.java          |   3 +-
 .../streaming/api/datastream/DataStream.java    |  66 ++++++++++-
 .../api/datastream/GroupedDataStream.java       |  70 ++++++++++--
 .../ComparableAggregationFunction.java          |  64 +++++++++++
 .../StreamingAggregationFunction.java           |  45 ++++++++
 .../StreamingMaxAggregationFunction.java        |  32 ++++++
 .../StreamingMinAggregationFunction.java        |  32 ++++++
 .../StreamingSumAggregationFunction.java        |  64 +++++++++++
 .../streaming/api/AggregationFunctionTest.java  | 109 +++++++++++++++++++
 .../examples/wordcount/WordCountLocal.java      |  25 +----
 12 files changed, 510 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 2ae765e..06a1308 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -22,7 +22,6 @@ 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.examples.function.JSONParseFlatMap;
-import org.apache.flink.streaming.examples.wordcount.WordCountLocal.WordCountCounter;
 import org.apache.flink.util.Collector;
 import org.apache.sling.commons.json.JSONException;
 
@@ -90,7 +89,7 @@ public class TwitterLocal {
 					}
 				})
 				.groupBy(0)
-				.reduce(new WordCountCounter());
+				.sum(1);
 
 		dataStream.print();
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/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 7973324..cd54a54 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
@@ -48,6 +48,7 @@ import org.apache.flink.streaming.api.streamcomponent.StreamTask;
 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.types.TypeInformation;
 
 /**
  * Object for building Apache Flink stream processing job graphs
@@ -322,6 +323,7 @@ public class JobGraphBuilder {
 		iterationIds.put(componentName, iterationID);
 		iterationIDtoSinkName.put(iterationID, componentName);
 		setBytesFrom(iterationTail, componentName);
+		//setInTypeWrappersFrom(iterationTail, componentName);
 		iterationWaitTime.put(iterationIDtoSinkName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
@@ -602,11 +604,43 @@ public class JobGraphBuilder {
 		operatorNames.put(to, operatorNames.get(from));
 		serializedFunctions.put(to, serializedFunctions.get(from));
 
+		setTypeWrappersFrom(from, to);
+	}
+
+	public void setTypeWrappersFrom(String from, String to) {
+		setInToOutTypeWrappersFrom(from, to);
+		setOutToOutTypeWrappersFrom(from, to);
+	}
+
+	public void setInToOutTypeWrappersFrom(String from, String to) {
+		//TODO rename function
 		typeWrapperIn1.put(to, typeWrapperOut1.get(from));
 		typeWrapperIn2.put(to, typeWrapperOut2.get(from));
+	}
+	
+	public void setOutToOutTypeWrappersFrom(String from, String to) {
+		//TODO rename function
 		typeWrapperOut1.put(to, typeWrapperOut1.get(from));
 		typeWrapperOut2.put(to, typeWrapperOut2.get(from));
 	}
+	
+	public void setInToInTypeWrappersFrom(String from, String to) {
+		//TODO rename function
+		typeWrapperIn1.put(to, typeWrapperIn1.get(from));
+		typeWrapperIn2.put(to, typeWrapperIn2.get(from));
+	}
+	
+	public TypeInformation<?> getInTypeInfo(String id){
+		// TODO 
+		System.out.println("DEBUG TypeInfo " + typeWrapperIn1.get(id));
+		return typeWrapperIn1.get(id).getTypeInfo();
+	}
+	
+	public TypeInformation<?> getOutTypeInfo(String id){
+		// TODO 
+		return typeWrapperOut1.get(id).getTypeInfo();
+	}
+	
 
 	/**
 	 * Sets instance sharing between the given components

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index ce4069e..54cab72 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -59,6 +59,7 @@ public class StreamCollector<OUT> implements Collector<OUT> {
 	public StreamCollector(int channelID,
 			SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
 		this.serializationDelegate = serializationDelegate;
+		
 		if (serializationDelegate != null) {
 			this.streamRecord = serializationDelegate.getInstance();
 		} else {
@@ -145,4 +146,4 @@ public class StreamCollector<OUT> implements Collector<OUT> {
 	@Override
 	public void close() {
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/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 64a07b5..9375762 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
@@ -36,6 +36,10 @@ import org.apache.flink.api.java.functions.RichMapFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.JobGraphBuilder;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
 import org.apache.flink.streaming.api.function.sink.PrintSinkFunction;
 import org.apache.flink.streaming.api.function.sink.SinkFunction;
 import org.apache.flink.streaming.api.function.sink.WriteFormatAsCsv;
@@ -60,6 +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.TypeSerializerWrapper;
+import org.apache.flink.types.TypeInformation;
 
 /**
  * A DataStream represents a stream of elements of the same type. A DataStream
@@ -465,6 +470,60 @@ public abstract class DataStream<OUT> {
 	}
 
 	/**
+	 * Applies an aggregation that sums the data stream at the given
+	 * position.
+	 * 
+	 * @param positionToSum
+	 *            The position in the data point to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> sum(int positionToSum) {
+		return aggregateAll(new StreamingSumAggregationFunction<OUT>(positionToSum));
+	}
+	
+	/**
+	 * Applies an aggregation that that gives the minimum of the data stream at the given
+	 * position.
+	 * 
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min(int positionToMin) {
+		return aggregateAll(new StreamingMinAggregationFunction<OUT>(positionToMin));
+	}
+	
+	/**
+	 * Applies an aggregation that gives the maximum of the data stream at the given
+	 * position.
+	 * 
+	 * @param positionToMax
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max(int positionToMax) {
+		return aggregateAll(new StreamingMaxAggregationFunction<OUT>(positionToMax));
+	}
+
+	private SingleOutputStreamOperator<OUT, ?> aggregateAll(StreamingAggregationFunction<OUT> aggregate) {
+		return aggregate(aggregate, new StreamReduceInvokable<OUT>(aggregate), "reduce");
+	}
+	
+	SingleOutputStreamOperator<OUT, ?> aggregate(StreamingAggregationFunction<OUT> aggregate, StreamReduceInvokable<OUT> invokable, String functionName) {
+		DataStream<OUT> inputStream = this.copy();
+		TypeInformation<?> info = this.jobGraphBuilder.getOutTypeInfo(inputStream.getId());
+
+		aggregate.setType(info);
+
+		SingleOutputStreamOperator<OUT, ?> returnStream = inputStream.addFunction(functionName,
+				aggregate, null, null, invokable);
+
+		this.jobGraphBuilder.setTypeWrappersFrom(inputStream.getId(), returnStream.getId());
+
+		return returnStream;
+	}
+
+	/**
 	 * Applies a Filter transformation on a {@link DataStream}. The
 	 * transformation calls a {@link FilterFunction} for each element of the
 	 * DataStream and retains only those element for which the function returns
@@ -497,7 +556,7 @@ public abstract class DataStream<OUT> {
 		PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>();
 		DataStreamSink<OUT> returnStream = addSink(inputStream, printFunction, null);
 
-		jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+		jobGraphBuilder.setInToOutTypeWrappersFrom(inputStream.getId(), returnStream.getId());
 
 		return returnStream;
 	}
@@ -853,9 +912,8 @@ public abstract class DataStream<OUT> {
 	 *            type of the return stream
 	 * @return the data stream constructed
 	 */
-	protected <R> SingleOutputStreamOperator<R, ?> addFunction(
-			String functionName, final Function function,
-			TypeSerializerWrapper<OUT> inTypeWrapper,
+	protected <R> SingleOutputStreamOperator<R, ?> addFunction(String functionName,
+			final Function function, TypeSerializerWrapper<OUT> inTypeWrapper,
 			TypeSerializerWrapper<R> outTypeWrapper,
 			StreamOperatorInvokable<OUT, R> functionInvokable) {
 		DataStream<OUT> inputStream = this.copy();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/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 4d0265a..94d6c8d 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
@@ -19,7 +19,12 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
 import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
@@ -27,6 +32,8 @@ 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.jasper.compiler.Node.ParamsAction;
+
 /**
  * A GroupedDataStream represents a data stream which has been partitioned by
  * the given key in the values. Operators like {@link #reduce},
@@ -46,12 +53,12 @@ public class GroupedDataStream<OUT> {
 	}
 
 	/**
-	 * Applies a reduce transformation on the grouped data stream grouped on by
-	 * the given key position. The {@link ReduceFunction} will receive input
-	 * values based on the key value. Only input values with the same key will
-	 * go to the same reducer.The user can also extend
-	 * {@link RichReduceFunction} to gain access to other features provided by
-	 * the {@link RichFuntion} interface.
+	 * Applies a reduce transformation on the grouped data stream grouped by the
+	 * given key position. The {@link ReduceFunction} will receive input values
+	 * based on the key value. Only input values with the same key will go to
+	 * the same reducer.The user can also extend {@link RichReduceFunction} to
+	 * gain access to other features provided by the {@link RichFuntion}
+	 * interface.
 	 * 
 	 * @param reducer
 	 *            The {@link ReduceFunction} that will be called for every
@@ -160,17 +167,16 @@ public class GroupedDataStream<OUT> {
 			long windowSize, long slideInterval) {
 		return windowReduce(reducer, windowSize, slideInterval, new DefaultTimestamp<OUT>());
 	}
-	
+
 	/**
 	 * Applies a group reduce transformation on preset "time" chunks of the
 	 * grouped data stream in a sliding window fashion. The
 	 * {@link GroupReduceFunction} will receive input values based on the key
 	 * value. Only input values with the same key will go to the same reducer.
 	 * When the reducer has ran for all the values in the batch, the window is
-	 * shifted forward. The time is determined by a
-	 * user-defined timestamp. The user can also extend {@link RichGroupReduceFunction}
-	 * to gain access to other features provided by the {@link RichFuntion}
-	 * interface.
+	 * shifted forward. The time is determined by a user-defined timestamp. The
+	 * user can also extend {@link RichGroupReduceFunction} to gain access to
+	 * other features provided by the {@link RichFuntion} interface.
 	 *
 	 * @param reducer
 	 *            The GroupReduceFunction that is called for each time window.
@@ -191,4 +197,46 @@ public class GroupedDataStream<OUT> {
 				windowSize, slideInterval, keyPosition, timestamp));
 	}
 
+	/**
+	 * Applies an aggregation that sums the grouped data stream at the given
+	 * position, grouped by the given key position. Input values with the same
+	 * key will be summed.
+	 * 
+	 * @param positionToSum
+	 *            The position in the data point to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> sum(final int positionToSum) {
+		return aggregateGroup(new StreamingSumAggregationFunction<OUT>(positionToSum));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum of the grouped data stream
+	 * at the given position, grouped by the given key position. Input values
+	 * with the same key will be minimized.
+	 * 
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> min(final int positionToMin) {
+		return aggregateGroup(new StreamingMinAggregationFunction<OUT>(positionToMin));
+	}
+
+	/**
+	 * 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.
+	 * 
+	 * @param positionToMax
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<OUT, ?> max(final int positionToMax) {
+		return aggregateGroup(new StreamingMaxAggregationFunction<OUT>(positionToMax));
+	}
+
+	private SingleOutputStreamOperator<OUT, ?> aggregateGroup(StreamingAggregationFunction<OUT> aggregate) {
+		return this.dataStream.aggregate(aggregate, new GroupReduceInvokable<OUT>(aggregate, keyPosition), "groupReduce");
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
new file mode 100644
index 0000000..dc74715
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregationFunction.java
@@ -0,0 +1,64 @@
+/**
+ * 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 abstract class ComparableAggregationFunction<T> extends StreamingAggregationFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	public ComparableAggregationFunction(int positionToAggregate) {
+		super(positionToAggregate);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public T reduce(T value1, T value2) throws Exception {
+		if (value1 instanceof Tuple) {
+			Tuple t1 = (Tuple) value1;
+			Tuple t2 = (Tuple) value2;
+
+			compare(t1, t2);
+
+			return (T) returnTuple;
+		} else if (value1 instanceof Comparable) {
+			if (isExtremal((Comparable<Object>) value1, value2)) {
+				value2 = value1;
+			}
+		} else {
+			throw new RuntimeException("The values " + value1 +  " and "+ value2 + " cannot be compared.");
+		}
+
+		return null;
+	}
+
+	public <R> void compare(Tuple tuple1, Tuple tuple2) throws InstantiationException,
+			IllegalAccessException {
+		copyTuple(tuple2);
+
+		Comparable<R> o1 = tuple1.getField(position);
+		R o2 = tuple2.getField(position);
+
+		if (isExtremal(o1, o2)) {
+			returnTuple.setField(o1, position);
+		}
+	}
+
+	public abstract <R> boolean isExtremal(Comparable<R> o1, R o2);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.java
new file mode 100644
index 0000000..42c1053
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingAggregationFunction.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.function.aggregation;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.types.TypeInformation;
+
+public abstract class StreamingAggregationFunction<T> implements ReduceFunction<T> {
+	private static final long serialVersionUID = 1L;
+	
+	protected int position;
+	private TypeSerializer<Tuple> typeSerializer;
+	protected Tuple returnTuple;
+
+	public StreamingAggregationFunction(int pos) {
+		this.position = pos;
+	}
+
+	@SuppressWarnings("unchecked")
+	public void setType(TypeInformation<?> type) {
+		this.typeSerializer = (TypeSerializer<Tuple>) type.createSerializer();
+	}
+
+	protected void copyTuple(Tuple tuple) throws InstantiationException, IllegalAccessException {
+		returnTuple = (Tuple) typeSerializer.createInstance();
+		typeSerializer.copy(tuple, returnTuple);
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
new file mode 100644
index 0000000..1a043c1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/StreamingSumAggregationFunction.java
@@ -0,0 +1,64 @@
+/**
+ * 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 StreamingSumAggregationFunction<T> extends StreamingAggregationFunction<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	public StreamingSumAggregationFunction(int pos) {
+		super(pos);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public T reduce(T value1, T value2) throws Exception {
+		if (value1 instanceof Tuple) {
+			Tuple tuple1 = (Tuple) value1;
+			Tuple tuple2 = (Tuple) value2;
+
+			copyTuple(tuple2);
+			returnTuple.setField(add(tuple1.getField(position), tuple2.getField(position)), position);
+
+			return (T) returnTuple;
+		} else {
+			return (T) add(value1, value2);
+		}
+	}
+
+	private Object add(Object value1, Object value2) {
+		if (value1 instanceof Integer) {
+			return (Integer) value1 + (Integer) value2;
+		} else if (value1 instanceof Double) {
+			return (Double) value1 + (Double) value2;
+		} else if (value1 instanceof Float) {
+			return (Float) value1 + (Float) value2;
+		} else if (value1 instanceof Long) {
+			return (Long) value1 + (Long) value2;
+		} else if (value1 instanceof Short) {
+			return (short) ((Short) value1 + (Short) value2);
+		} else if (value1 instanceof Byte) {
+			return (byte) ((Byte) value1 + (Byte) value2);
+		} else {
+			throw new RuntimeException("DataStream cannot be summed because the class "
+					+ value1.getClass().getSimpleName() + " does not support the + operator.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/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
new file mode 100644
index 0000000..7a502aa
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMaxAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingMinAggregationFunction;
+import org.apache.flink.streaming.api.function.aggregation.StreamingSumAggregationFunction;
+import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.junit.Test;
+
+public class AggregationFunctionTest {
+
+	@Test
+	public void groupSumIntegerTest() {
+		ArrayList<Tuple2<Integer, Integer>> inputList = new ArrayList<Tuple2<Integer, Integer>>();
+
+		List<Tuple2<Integer, Integer>> expectedSumList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Tuple2<Integer, Integer>> expectedMinList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Tuple2<Integer, Integer>> expectedMaxList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Tuple2<Integer, Integer>> expectedGroupSumList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Tuple2<Integer, Integer>> expectedGroupMinList = new ArrayList<Tuple2<Integer, Integer>>();
+		List<Tuple2<Integer, Integer>> expectedGroupMaxList = new ArrayList<Tuple2<Integer, Integer>>();
+
+		int groupedSum0 = 0;
+		int groupedSum1 = 0;
+		int groupedSum2 = 0;
+
+		for (int i = 0; i < 9; i++) {
+			inputList.add(new Tuple2<Integer, Integer>(i % 3, i));
+
+			expectedSumList.add(new Tuple2<Integer, Integer>(i % 3, (i + 1) * i / 2));
+			expectedMinList.add(new Tuple2<Integer, Integer>(i % 3, 0));
+			expectedMaxList.add(new Tuple2<Integer, Integer>(i % 3, i));
+
+			int groupedSum;
+			switch (i % 3) {
+			case 0:
+				groupedSum = groupedSum0 += i;
+				break;
+			case 1:
+				groupedSum = groupedSum1 += i;
+				break;
+			default:
+				groupedSum = groupedSum2 += i;
+				break;
+			}
+
+			expectedGroupSumList.add(new Tuple2<Integer, Integer>(i % 3, groupedSum));
+			expectedGroupMinList.add(new Tuple2<Integer, Integer>(i % 3, i % 3));
+			expectedGroupMaxList.add(new Tuple2<Integer, Integer>(i % 3, i));
+		}
+
+		StreamingSumAggregationFunction<Tuple2<Integer, Integer>> sumFunction = new StreamingSumAggregationFunction<Tuple2<Integer, Integer>>(
+				1);
+		StreamingMinAggregationFunction<Tuple2<Integer, Integer>> minFunction = new StreamingMinAggregationFunction<Tuple2<Integer, Integer>>(
+				1);
+		StreamingMaxAggregationFunction<Tuple2<Integer, Integer>> maxFunction = new StreamingMaxAggregationFunction<Tuple2<Integer, Integer>>(
+				1);
+
+		sumFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
+		minFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
+		maxFunction.setType(TypeExtractor.getForObject(new Tuple2<Integer, Integer>(0, 0)));
+
+		List<Tuple2<Integer, Integer>> sumList = MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(sumFunction), inputList);
+		List<Tuple2<Integer, Integer>> minList = MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(minFunction), inputList);
+		List<Tuple2<Integer, Integer>> maxList = MockInvokable.createAndExecute(
+				new StreamReduceInvokable<Tuple2<Integer, Integer>>(maxFunction), inputList);
+
+		List<Tuple2<Integer, Integer>> groupedSumList = MockInvokable.createAndExecute(
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(sumFunction, 0), inputList);
+		List<Tuple2<Integer, Integer>> groupedMinList = MockInvokable.createAndExecute(
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(minFunction, 0), inputList);
+		List<Tuple2<Integer, Integer>> groupedMaxList = MockInvokable.createAndExecute(
+				new GroupReduceInvokable<Tuple2<Integer, Integer>>(maxFunction, 0), inputList);
+
+		assertEquals(expectedSumList, sumList);
+		assertEquals(expectedMinList, minList);
+		assertEquals(expectedMaxList, maxList);
+		assertEquals(expectedGroupSumList, groupedSumList);
+		assertEquals(expectedGroupMinList, groupedMinList);
+		assertEquals(expectedGroupMaxList, groupedMaxList);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0c8f1dac/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
index e554d91..f78cd1a 100644
--- 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
@@ -20,40 +20,29 @@ package org.apache.flink.streaming.examples.wordcount;
 import java.util.StringTokenizer;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
 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>> {
+	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 {
+		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 class WordCountCounter implements ReduceFunction<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
 
-		@Override
-		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
-				Tuple2<String, Integer> value2) throws Exception {
-			return new Tuple2<String, Integer>(value1.f0, value1.f1 + value2.f1);
-		}
-
-	}
-	
 	public static void main(String[] args) {
 
 		TestDataUtil.downloadIfNotExists("hamlet.txt");
@@ -61,10 +50,8 @@ public class WordCountLocal {
 
 		DataStream<Tuple2<String, Integer>> dataStream = env
 				.readTextFile("src/test/resources/testdata/hamlet.txt")
-				.flatMap(new WordCountSplitter())
-				.groupBy(0)
-				.reduce(new WordCountCounter());
-		
+				.flatMap(new WordCountSplitter()).groupBy(0).sum(1);
+
 		dataStream.print();
 
 		env.execute();


[13/18] git commit: [streaming] Added counter aggregation

Posted by mb...@apache.org.
[streaming] Added counter aggregation


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

Branch: refs/heads/master
Commit: 03a28cbb755c49c2b97205989434c0c1664e8a3e
Parents: b6ffdba
Author: ghermann <re...@gmail.com>
Authored: Mon Sep 15 17:25:22 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    | 26 +---------
 .../api/datastream/BatchedDataStream.java       |  3 +-
 .../streaming/api/datastream/DataStream.java    | 37 +++++++++------
 .../api/datastream/GroupedDataStream.java       |  3 +-
 .../invokable/operator/CounterInvokable.java    | 50 ++++++++++++++++++++
 .../operator/CounterInvokableTest.java          | 39 +++++++++++++++
 6 files changed, 115 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/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 4bb022a..e6c5042 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
@@ -312,10 +312,11 @@ public class JobGraphBuilder {
 			int parallelism, long waitTime) {
 
 		addComponent(componentName, StreamIterationSink.class, null, null, null, parallelism);
+
 		iterationIds.put(componentName, iterationID);
 		iterationIDtoSinkName.put(iterationID, componentName);
+
 		setBytesFrom(iterationTail, componentName);
-		// setInTypeWrappersFrom(iterationTail, componentName);
 		iterationWaitTime.put(iterationIDtoSinkName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
@@ -576,44 +577,21 @@ public class JobGraphBuilder {
 	 *            to
 	 */
 	public void setBytesFrom(String from, String to) {
-
 		operatorNames.put(to, operatorNames.get(from));
 		serializedFunctions.put(to, serializedFunctions.get(from));
 
-		setTypeWrappersFrom(from, to);
-	}
-
-	public void setTypeWrappersFrom(String from, String to) {
-		setInToOutTypeWrappersFrom(from, to);
-		setOutToOutTypeWrappersFrom(from, to);
-	}
-
-	public void setInToOutTypeWrappersFrom(String from, String to) {
-		// TODO rename function
 		typeWrapperIn1.put(to, typeWrapperOut1.get(from));
 		typeWrapperIn2.put(to, typeWrapperOut2.get(from));
-	}
-
-	public void setOutToOutTypeWrappersFrom(String from, String to) {
-		// TODO rename function
 		typeWrapperOut1.put(to, typeWrapperOut1.get(from));
 		typeWrapperOut2.put(to, typeWrapperOut2.get(from));
 	}
 
-	public void setInToInTypeWrappersFrom(String from, String to) {
-		// TODO rename function
-		typeWrapperIn1.put(to, typeWrapperIn1.get(from));
-		typeWrapperIn2.put(to, typeWrapperIn2.get(from));
-	}
-
 	public TypeInformation<?> getInTypeInfo(String id) {
-		// TODO
 		System.out.println("DEBUG TypeInfo " + typeWrapperIn1.get(id));
 		return typeWrapperIn1.get(id).getTypeInfo();
 	}
 
 	public TypeInformation<?> getOutTypeInfo(String id) {
-		// TODO
 		return typeWrapperOut1.get(id).getTypeInfo();
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/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 0aa5de6..bcedac9 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
@@ -202,9 +202,8 @@ public class BatchedDataStream<OUT> {
 		BatchReduceInvokable<OUT> invokable = getReduceInvokable(aggregate);
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = dataStream.addFunction("batchReduce",
-				aggregate, null, null, invokable);
+				aggregate, dataStream.outTypeWrapper, dataStream.outTypeWrapper, invokable);
 
-		dataStream.jobGraphBuilder.setTypeWrappersFrom(dataStream.getId(), returnStream.getId());
 		return returnStream;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/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 bebda91..0e1ae57 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,6 +47,7 @@ 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.operator.CounterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
@@ -60,6 +61,7 @@ import org.apache.flink.streaming.partitioner.ForwardPartitioner;
 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.types.TypeInformation;
 
@@ -86,7 +88,7 @@ public class DataStream<OUT> {
 	protected List<String> userDefinedNames;
 	protected boolean selectAll;
 	protected StreamPartitioner<OUT> partitioner;
-	protected TypeSerializerWrapper<OUT> outTypeWrapper;
+	protected final TypeSerializerWrapper<OUT> outTypeWrapper;
 	protected List<DataStream<OUT>> mergedStreams;
 
 	protected final JobGraphBuilder jobGraphBuilder;
@@ -556,15 +558,26 @@ public class DataStream<OUT> {
 	public SingleOutputStreamOperator<OUT, ?> max() {
 		return max(0);
 	}
+	
+	/**
+	 * Applies an aggregation that gives the count of the data point.
+	 * 
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<Long, ?> count() {
+		TypeSerializerWrapper<OUT> inTypeWrapper = outTypeWrapper;
+		TypeSerializerWrapper<Long> outTypeWrapper = new ObjectTypeWrapper<Long>(new Long(0));
+
+		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, null,
-				null, invokable);
+		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("reduce", aggregate, outTypeWrapper,
+				outTypeWrapper, invokable);
 
-		this.jobGraphBuilder.setTypeWrappersFrom(getId(), returnStream.getId());
 		return returnStream;
 	}
 
@@ -599,9 +612,7 @@ public class DataStream<OUT> {
 	public DataStreamSink<OUT> print() {
 		DataStream<OUT> inputStream = this.copy();
 		PrintSinkFunction<OUT> printFunction = new PrintSinkFunction<OUT>();
-		DataStreamSink<OUT> returnStream = addSink(inputStream, printFunction, null);
-
-		jobGraphBuilder.setInToOutTypeWrappersFrom(inputStream.getId(), returnStream.getId());
+		DataStreamSink<OUT> returnStream = addSink(inputStream, printFunction, outTypeWrapper);
 
 		return returnStream;
 	}
@@ -721,8 +732,7 @@ public class DataStream<OUT> {
 	private DataStreamSink<OUT> writeAsText(DataStream<OUT> inputStream, String path,
 			WriteFormatAsText<OUT> format, long millis, OUT endTuple) {
 		DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
-				path, format, millis, endTuple), null);
-		jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+				path, format, millis, endTuple), inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}
@@ -749,8 +759,7 @@ 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), null);
-		jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}
@@ -873,8 +882,7 @@ public class DataStream<OUT> {
 	private DataStreamSink<OUT> writeAsCsv(DataStream<OUT> inputStream, String path,
 			WriteFormatAsCsv<OUT> format, long millis, OUT endTuple) {
 		DataStreamSink<OUT> returnStream = addSink(inputStream, new WriteSinkFunctionByMillis<OUT>(
-				path, format, millis, endTuple));
-		jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+				path, format, millis, endTuple), inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}
@@ -901,8 +909,7 @@ 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), null);
-		jobGraphBuilder.setBytesFrom(inputStream.getId(), returnStream.getId());
+				new WriteSinkFunctionByBatches<OUT>(path, format, batchSize, endTuple), inputStream.outTypeWrapper);
 		jobGraphBuilder.setMutability(returnStream.getId(), false);
 		return returnStream;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/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 30826d3..e30d316 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
@@ -243,9 +243,8 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 		GroupedReduceInvokable<OUT> invokable = new GroupedReduceInvokable<OUT>(aggregate, keyPosition);
 
 		SingleOutputStreamOperator<OUT, ?> returnStream = addFunction("groupReduce", aggregate,
-				null, null, invokable);
+				outTypeWrapper, outTypeWrapper, invokable);
 
-		this.jobGraphBuilder.setTypeWrappersFrom(getId(), returnStream.getId());
 		return returnStream;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/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
new file mode 100644
index 0000000..29903b1
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokable.java
@@ -0,0 +1,50 @@
+/**
+ * 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.streaming.api.invokable.StreamOperatorInvokable;
+
+public class CounterInvokable<IN> extends StreamOperatorInvokable<IN, Long> {
+	private static final long serialVersionUID = 1L;
+	
+	Long count = 0L;
+	
+	public CounterInvokable() {
+		super(null);
+	}
+	
+	@Override
+	protected void immutableInvoke() throws Exception {
+		while ((reuse = recordIterator.next(reuse)) != null) {
+			callUserFunctionAndLogException();
+			resetReuse();
+		}
+	}
+
+	@Override
+	protected void mutableInvoke() throws Exception {
+		while ((reuse = recordIterator.next(reuse)) != null) {
+			callUserFunctionAndLogException();
+		}
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+		collector.collect(++count);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/03a28cbb/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokableTest.java
new file mode 100644
index 0000000..2124eb7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CounterInvokableTest.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.streaming.util.MockInvokable;
+import org.junit.Test;
+
+public class CounterInvokableTest {
+
+	@Test
+	public void counterTest() {
+		CounterInvokable<String> invokable = new CounterInvokable<String>();
+
+		List<Long> expected = Arrays.asList(1L, 2L, 3L);
+		List<Long> actual = MockInvokable.createAndExecute(invokable, Arrays.asList("one", "two", "three"));
+		
+		assertEquals(expected, actual);
+	}
+}


[18/18] git commit: [streaming] Updated logging to utilize SLF4J

Posted by mb...@apache.org.
[streaming] Updated logging to utilize SLF4J


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

Branch: refs/heads/master
Commit: d0dd5138fd0bddf2bf942bffee1681c298043b3e
Parents: 5f601cf
Author: ghermann <re...@gmail.com>
Authored: Wed Sep 10 11:58:36 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 .../flink-streaming-connectors/pom.xml          |  10 +-
 .../streaming/connectors/flume/FlumeSink.java   |   2 +-
 .../connectors/flume/FlumeTopology.java         |  30 +--
 .../connectors/json/JSONParseFlatMap.java       | 144 +++++++++++++
 .../streaming/connectors/json/JSONParser.java   | 175 +++++++++++++++
 .../connectors/kafka/KafkaTopology.java         |  12 +-
 .../streaming/connectors/rabbitmq/RMQSink.java  |   2 +-
 .../connectors/rabbitmq/RMQSource.java          |  12 +-
 .../connectors/rabbitmq/RMQTopology.java        |  15 +-
 .../connectors/twitter/TwitterLocal.java        |   2 +-
 .../connectors/twitter/TwitterSource.java       |  71 ++++---
 .../connectors/twitter/TwitterStreaming.java    |   8 +-
 .../connectors/json/JSONParserTest.java         |  74 +++++++
 .../connectors/json/JSONParserTest2.java        |  95 +++++++++
 .../flink/streaming/api/JobGraphBuilder.java    | 211 ++++++++-----------
 .../api/collector/DirectedStreamCollector.java  |  11 +-
 .../api/collector/StreamCollector.java          |   9 +-
 .../environment/RemoteStreamEnvironment.java    |   2 +-
 .../api/invokable/StreamOperatorInvokable.java  |   4 +-
 .../api/invokable/operator/co/CoInvokable.java  |  14 +-
 .../api/streamcomponent/OutputHandler.java      |  24 +--
 .../streamcomponent/StreamIterationSink.java    |  13 +-
 .../streamcomponent/StreamIterationSource.java  |   2 +-
 .../api/streamcomponent/StreamSink.java         |   4 +-
 .../flink/streaming/util/TestDataUtil.java      |   8 +-
 .../flink-streaming-examples/pom.xml            |   6 +
 .../examples/function/JSONParseFlatMap.java     | 144 -------------
 .../streaming/examples/function/JSONParser.java | 175 ---------------
 .../examples/function/JSONParserTest.java       |  73 -------
 .../examples/function/JSONParserTest2.java      |  94 ---------
 30 files changed, 714 insertions(+), 732 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
index ee99d7f..00fc675 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml
@@ -43,12 +43,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-examples</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
 			<groupId>org.apache.kafka</groupId>
 			<artifactId>kafka_2.10</artifactId>
 			<version>0.8.0</version>
@@ -65,6 +59,10 @@ under the License.
 					<groupId>log4j</groupId>
 					<artifactId>log4j</artifactId>
 				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-simple</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
index c4618a7..6bc5d8a 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -110,7 +110,7 @@ public abstract class FlumeSink<IN> implements SinkFunction<IN> {
 						Thread.sleep(1000);
 					} catch (InterruptedException e1) {
 						if (LOG.isErrorEnabled()) {
-							LOG.error("Interrupted while trying to connect " + port + " at " + host);
+							LOG.error("Interrupted while trying to connect {} at {}", port, host);
 						}
 					}
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index 38ea6ef..73668c6 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -18,14 +18,15 @@
 package org.apache.flink.streaming.connectors.flume;
 
 import org.apache.commons.lang.SerializationUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 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;
 
 public class FlumeTopology {
-	private static final Log LOG = LogFactory.getLog(FlumeTopology.class);
+	private static final Logger LOG = LoggerFactory.getLogger(FlumeTopology.class);
+
 	public static class MyFlumeSink extends FlumeSink<String> {
 		private static final long serialVersionUID = 1L;
 
@@ -39,8 +40,8 @@ public class FlumeTopology {
 				try {
 					sendAndClose();
 				} catch (Exception e) {
-					throw new RuntimeException("Error while closing Flume connection with " + port + " at "
-							+ host, e);
+					throw new RuntimeException("Error while closing Flume connection with " + port
+							+ " at " + host, e);
 				}
 			}
 			return SerializationUtils.serialize(tuple);
@@ -53,12 +54,13 @@ public class FlumeTopology {
 
 		@Override
 		public void invoke(String value) {
-			LOG.info("String: <" + value + "> arrived from Flume");
-			
+			if (LOG.isInfoEnabled()) {
+				LOG.info("String: <{}> arrived from Flume", value);
+			}
 		}
-		
+
 	}
-	
+
 	public static class MyFlumeSource extends FlumeSource<String> {
 		private static final long serialVersionUID = 1L;
 
@@ -82,14 +84,12 @@ public class FlumeTopology {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 
 		@SuppressWarnings("unused")
-		DataStream<String> dataStream1 = env
-			.addSource(new MyFlumeSource("localhost", 41414))
-			.addSink(new MyFlumePrintSink());
+		DataStream<String> dataStream1 = env.addSource(new MyFlumeSource("localhost", 41414))
+				.addSink(new MyFlumePrintSink());
 
 		@SuppressWarnings("unused")
-		DataStream<String> dataStream2 = env
-			.fromElements("one", "two", "three", "four", "five", "q")
-			.addSink(new MyFlumeSink("localhost", 42424));
+		DataStream<String> dataStream2 = env.fromElements("one", "two", "three", "four", "five",
+				"q").addSink(new MyFlumeSink("localhost", 42424));
 
 		env.execute();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
new file mode 100644
index 0000000..96b1bf7
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
@@ -0,0 +1,144 @@
+/**
+ * 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.connectors.json;
+
+import org.apache.flink.api.java.functions.RichFlatMapFunction;
+import org.apache.sling.commons.json.JSONException;
+
+/**
+ * Abstract class derived from {@link RichFlatMapFunction} to handle JSON files.
+ * 
+ * @param <IN>
+ *            Type of the input elements.
+ * @param <OUT>
+ *            Type of the returned elements.
+ */
+public abstract class JSONParseFlatMap<IN, OUT> extends RichFlatMapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	// private static final Log LOG = LogFactory.getLog(JSONParseFlatMap.class);
+
+	/**
+	 * Get the value object associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public Object get(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).get("retValue");
+	}
+
+	/**
+	 * Get the boolean value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public boolean getBoolean(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getBoolean("retValue");
+	}
+
+	/**
+	 * Get the double value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public double getDouble(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getDouble("retValue");
+	}
+
+	/**
+	 * Get the int value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public int getInt(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getInt("retValue");
+	}
+
+	/**
+	 * Get the long value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public long getLong(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getLong("retValue");
+	}
+	
+	/**
+	 * Get the String value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public String getString(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+		
+		return parser.parse(field).getString("retValue");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
new file mode 100644
index 0000000..4e34483
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
@@ -0,0 +1,175 @@
+/**
+ * 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.connectors.json;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.StringTokenizer;
+
+import org.apache.sling.commons.json.JSONArray;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+
+/**
+ * A JSONParser contains a JSONObject and provides opportunity to access
+ * embedded fields in JSON code.
+ */
+public class JSONParser {
+
+	private JSONObject originalJO;
+	private String searchedfield;
+	private Object temp;
+
+	/**
+	 * Construct a JSONParser from a string. The string has to be a JSON code
+	 * from which we want to get a field.
+	 * 
+	 * @param jsonText
+	 *            A string which contains a JSON code. String representation of
+	 *            a JSON code.
+	 * @throws JSONException
+	 *             If there is a syntax error in the source string.
+	 */
+	public JSONParser(String jsonText) throws JSONException {
+		originalJO = new JSONObject(jsonText);
+	}
+
+	/**
+	 * 
+	 * Parse the JSON code passed to the constructor to find the given key.
+	 * 
+	 * @param key
+	 *            The key whose value is searched for.
+	 * @return A JSONObject which has only one field called "retValue" and the
+	 *         value associated to it is the searched value. The methods of
+	 *         JSONObject can be used to get the field value in a desired
+	 *         format.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	public JSONObject parse(String key) throws JSONException {
+		initializeParser(key);
+		parsing();
+		return putResultInJSONObj();
+	}
+
+	/**
+	 * Prepare the fields of the class for the parsing
+	 * 
+	 * @param key
+	 *            The key whose value is searched for.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void initializeParser(String key) throws JSONException {
+		searchedfield = key;
+		temp = new JSONObject(originalJO.toString());
+	}
+
+	/**
+	 * This function goes through the given field and calls the appropriate
+	 * functions to treat the units between the punctuation marks.
+	 * 
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void parsing() throws JSONException {
+		StringTokenizer st = new StringTokenizer(searchedfield, ".");
+		while (st.hasMoreTokens()) {
+			find(st.nextToken());
+		}
+	}
+
+	/**
+	 * Search for the next part of the field and update the state if it was
+	 * found.
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void find(String nextToken) throws JSONException {
+		if (endsWithBracket(nextToken)) {
+			treatAllBracket(nextToken);
+		} else {
+			temp = ((JSONObject) temp).get(nextToken);
+		}
+	}
+
+	/**
+	 * Determine whether the given string ends with a closing square bracket ']'
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @return True if the given string ends with a closing square bracket ']'
+	 *         and false otherwise.
+	 */
+	private boolean endsWithBracket(String nextToken) {
+		return nextToken.substring(nextToken.length() - 1).endsWith("]");
+	}
+
+	/**
+	 * Handle (multidimensional) arrays. Treat the square bracket pairs one
+	 * after the other if necessary.
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @throws JSONException
+	 *             If the searched element is not found.
+	 */
+	private void treatAllBracket(String nextToken) throws JSONException {
+		List<String> list = Arrays.asList(nextToken.split("\\["));
+		ListIterator<String> iter = list.listIterator();
+
+		temp = ((JSONObject) temp).get(iter.next());
+
+		while (iter.hasNext()) {
+			int index = Integer.parseInt(cutBracket(iter.next()));
+			temp = ((JSONArray) temp).get(index);
+		}
+	}
+
+	/**
+	 * Remove the last character of the string.
+	 * 
+	 * @param string
+	 *            String to modify.
+	 * @return The given string without the last character.
+	 */
+	private String cutBracket(String string) {
+		return string.substring(0, string.length() - 1);
+	}
+
+	/**
+	 * Save the result of the search into a JSONObject.
+	 * 
+	 * @return A special JSONObject which contain only one key. The value
+	 *         associated to this key is the result of the search.
+	 * @throws JSONException
+	 *             If there is a problem creating the JSONObject. (e.g. invalid
+	 *             syntax)
+	 */
+	private JSONObject putResultInJSONObj() throws JSONException {
+		JSONObject jo = new JSONObject();
+		jo.put("retValue", temp);
+		return jo;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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 2c89471..64ea810 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
@@ -18,16 +18,16 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 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;
 import org.apache.flink.streaming.api.function.source.SourceFunction;
 import org.apache.flink.util.Collector;
-import org.apache.flink.streaming.api.function.sink.SinkFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class KafkaTopology {
-	private static final Log LOG = LogFactory.getLog(KafkaTopology.class);
+	private static final Logger LOG = LoggerFactory.getLogger(KafkaTopology.class);
 	
 	public static final class MySource implements SourceFunction<Tuple1<String>> {
 		private static final long serialVersionUID = 1L;
@@ -82,7 +82,9 @@ public class KafkaTopology {
 
 		@Override
 		public void invoke(Tuple1<String> value) {
-			LOG.info("String: " + value + " arrived from Kafka");
+			if (LOG.isInfoEnabled()) {
+				LOG.info("String: <{}> arrived from Kafka", value);
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index ae04298..22e9aae 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -84,7 +84,7 @@ public abstract class RMQSink<IN> implements SinkFunction<IN> {
 			}
 		} catch (IOException e) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error("Cannot send RMQ message " + QUEUE_NAME + " at " + HOST_NAME);
+				LOG.error("Cannot send RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 1fcd57b..4f2feba 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -19,11 +19,11 @@ package org.apache.flink.streaming.connectors.rabbitmq;
 
 import java.io.IOException;
 
-import net.spy.memcached.compat.log.Logger;
-import net.spy.memcached.compat.log.LoggerFactory;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.function.source.RichSourceFunction;
 import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.rabbitmq.client.Channel;
 import com.rabbitmq.client.Connection;
@@ -86,7 +86,9 @@ public abstract class RMQSource<OUT> extends RichSourceFunction<OUT> {
 			try {
 				delivery = consumer.nextDelivery();
 			} catch (Exception e) {
-				LOG.error("Cannot receive RMQ message " + QUEUE_NAME + " at " + HOST_NAME);
+				if (LOG.isErrorEnabled()) {
+					LOG.error("Cannot recieve RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
+				}
 			}
 
 			outTuple = deserialize(delivery.getBody());
@@ -103,8 +105,8 @@ public abstract class RMQSource<OUT> extends RichSourceFunction<OUT> {
 		try {
 			connection.close();
 		} catch (IOException e) {
-			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME + " at "
-					+ HOST_NAME, e);
+			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME
+					+ " at " + HOST_NAME, e);
 		}
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
index 6cdda17..6d343dc 100755
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -18,15 +18,15 @@
 package org.apache.flink.streaming.connectors.rabbitmq;
 
 import org.apache.commons.lang.SerializationUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.flink.streaming.api.datastream.DataStream;
+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;
+import org.apache.flink.streaming.api.function.sink.SinkFunction;
 import org.apache.flink.streaming.connectors.kafka.KafkaTopology;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class RMQTopology {
-	private static final Log LOG = LogFactory.getLog(KafkaTopology.class);
+	private static final Logger LOG = LoggerFactory.getLogger(KafkaTopology.class);
 	
 	public static final class MyRMQSink extends RMQSink<String> {
 		public MyRMQSink(String HOST_NAME, String QUEUE_NAME) {
@@ -50,8 +50,9 @@ public class RMQTopology {
 
 		@Override
 		public void invoke(String value) {
-			LOG.info("String: <" + value + "> arrived from RMQ");
-			
+			if (LOG.isInfoEnabled()) {
+				LOG.info("String: <{}> arrived from RMQ", value);
+			}
 		}
 		
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
index 06a1308..465a500 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterLocal.java
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.MapFunction;
 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.examples.function.JSONParseFlatMap;
+import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
 import org.apache.flink.util.Collector;
 import org.apache.sling.commons.json.JSONException;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index 4aa7a43..525f4c8 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -41,8 +41,8 @@ import com.twitter.hbc.httpclient.auth.Authentication;
 import com.twitter.hbc.httpclient.auth.OAuth1;
 
 /**
- * Implementation of {@link SourceFunction} specialized to emit tweets from Twitter.
- * It can connect to Twitter Streaming API, collect tweets and 
+ * Implementation of {@link SourceFunction} specialized to emit tweets from
+ * Twitter. It can connect to Twitter Streaming API, collect tweets and
  */
 public class TwitterSource extends RichSourceFunction<String> {
 
@@ -60,8 +60,10 @@ public class TwitterSource extends RichSourceFunction<String> {
 
 	/**
 	 * Create {@link TwitterSource} for streaming
+	 * 
 	 * @param authPath
-	 * Location of the properties file containing the required authentication information. 
+	 *            Location of the properties file containing the required
+	 *            authentication information.
 	 */
 	public TwitterSource(String authPath) {
 		this.authPath = authPath;
@@ -69,10 +71,11 @@ public class TwitterSource extends RichSourceFunction<String> {
 	}
 
 	/**
-	 * Create {@link TwitterSource} to 
-	 * collect finite number of tweets
+	 * Create {@link TwitterSource} to collect finite number of tweets
+	 * 
 	 * @param authPath
-	 * Location of the properties file containing the required authentication information. 
+	 *            Location of the properties file containing the required
+	 *            authentication information.
 	 * @param numberOfTweets
 	 * 
 	 */
@@ -86,17 +89,17 @@ public class TwitterSource extends RichSourceFunction<String> {
 	public void open(Configuration parameters) throws Exception {
 		initializeConnection();
 	}
-	
+
 	@Override
 	public void invoke(Collector<String> collector) throws Exception {
-		
+
 		if (streaming) {
 			collectMessages(collector);
 		} else {
 			collectFiniteMessages(collector);
 		}
 	}
-	
+
 	@Override
 	public void close() throws Exception {
 		closeConnection();
@@ -136,9 +139,9 @@ public class TwitterSource extends RichSourceFunction<String> {
 	}
 
 	/**
-	 * Reads the given properties file for the authentication data.   
-	 * @return
-	 * the authentication data.
+	 * Reads the given properties file for the authentication data.
+	 * 
+	 * @return the authentication data.
 	 */
 	private Properties loadAuthenticationProperties() {
 		Properties properties = new Properties();
@@ -147,18 +150,15 @@ public class TwitterSource extends RichSourceFunction<String> {
 			properties.load(input);
 			input.close();
 		} catch (IOException ioe) {
-			new RuntimeException("Cannot open .properties file: " + authPath,
-					ioe);
+			new RuntimeException("Cannot open .properties file: " + authPath, ioe);
 		}
 		return properties;
 	}
 
-	private void initializeClient(StatusesSampleEndpoint endpoint,
-			Authentication auth) {
+	private void initializeClient(StatusesSampleEndpoint endpoint, Authentication auth) {
 
-		client = new ClientBuilder().name("twitterSourceClient")
-				.hosts(Constants.STREAM_HOST).endpoint(endpoint)
-				.authentication(auth)
+		client = new ClientBuilder().name("twitterSourceClient").hosts(Constants.STREAM_HOST)
+				.endpoint(endpoint).authentication(auth)
 				.processor(new StringDelimitedProcessor(queue)).build();
 
 		client.connect();
@@ -166,8 +166,9 @@ public class TwitterSource extends RichSourceFunction<String> {
 
 	/**
 	 * Put tweets into collector
+	 * 
 	 * @param collector
-	 * Collector in which the tweets are collected.
+	 *            Collector in which the tweets are collected.
 	 */
 	protected void collectFiniteMessages(Collector<String> collector) {
 
@@ -186,8 +187,9 @@ public class TwitterSource extends RichSourceFunction<String> {
 
 	/**
 	 * Put tweets into collector
+	 * 
 	 * @param collector
-	 * Collector in which the tweets are collected.
+	 *            Collector in which the tweets are collected.
 	 */
 	protected void collectMessages(Collector<String> collector) {
 
@@ -202,14 +204,15 @@ public class TwitterSource extends RichSourceFunction<String> {
 
 	/**
 	 * Put one tweet into the collector.
+	 * 
 	 * @param collector
-	 * Collector in which the tweets are collected.
+	 *            Collector in which the tweets are collected.
 	 */
 	protected void collectOneMessage(Collector<String> collector) {
 		if (client.isDone()) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error("Client connection closed unexpectedly: "
-						+ client.getExitEvent().getMessage());
+				LOG.error("Client connection closed unexpectedly: {}", client.getExitEvent()
+						.getMessage());
 			}
 		}
 
@@ -219,8 +222,7 @@ public class TwitterSource extends RichSourceFunction<String> {
 				collector.collect(msg);
 			} else {
 				if (LOG.isInfoEnabled()) {
-					LOG.info("Did not receive a message in " + waitSec
-							+ " seconds");
+					LOG.info("Did not receive a message in {} seconds", waitSec);
 				}
 			}
 		} catch (InterruptedException e) {
@@ -243,7 +245,8 @@ public class TwitterSource extends RichSourceFunction<String> {
 	}
 
 	/**
-	 * Get the size of the queue in which the tweets are contained temporarily. 
+	 * Get the size of the queue in which the tweets are contained temporarily.
+	 * 
 	 * @return
 	 */
 	public int getQueueSize() {
@@ -251,18 +254,19 @@ public class TwitterSource extends RichSourceFunction<String> {
 	}
 
 	/**
-	 * Set the size of the queue in which the tweets are contained temporarily. 
+	 * Set the size of the queue in which the tweets are contained temporarily.
+	 * 
 	 * @param queueSize
-	 * The desired value.
+	 *            The desired value.
 	 */
 	public void setQueueSize(int queueSize) {
 		this.queueSize = queueSize;
 	}
-	
+
 	/**
 	 * This function tells how long TwitterSource waits for the tweets.
-	 * @return
-	 * Number of second.
+	 * 
+	 * @return Number of second.
 	 */
 	public int getWaitSec() {
 		return waitSec;
@@ -270,8 +274,9 @@ public class TwitterSource extends RichSourceFunction<String> {
 
 	/**
 	 * This function sets how long TwitterSource should wait for the tweets.
+	 * 
 	 * @param waitSec
-	 * The desired value.
+	 *            The desired value.
 	 */
 	public void setWaitSec(int waitSec) {
 		this.waitSec = waitSec;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
index b715d18..5927ce4 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -17,15 +17,15 @@
 
 package org.apache.flink.streaming.connectors.twitter;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.flink.api.java.tuple.Tuple5;
 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;
-import org.apache.flink.streaming.examples.function.JSONParseFlatMap;
+import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
 import org.apache.flink.util.Collector;
 import org.apache.sling.commons.json.JSONException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TwitterStreaming {
 
@@ -33,7 +33,7 @@ public class TwitterStreaming {
 	private static final int SOURCE_PARALLELISM = 1;
 	private static final int NUMBEROFTWEETS = 100;
 
-	private static final Log LOG = LogFactory.getLog(TwitterStreaming.class);
+	private static final Logger LOG = LoggerFactory.getLogger(TwitterStreaming.class);
 
 	public static class TwitterSink implements SinkFunction<Tuple5<Long, Integer, String, String, String>> {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
new file mode 100644
index 0000000..b84c852
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
@@ -0,0 +1,74 @@
+/**
+ * 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.connectors.json;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.flink.streaming.connectors.json.JSONParser;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class JSONParserTest {
+
+	private String jsonText;
+	private String searchedField;
+
+	public JSONParserTest(String text, String field) {
+		jsonText = text;
+		searchedField = field;
+	}
+
+	@Parameters
+	public static Collection<Object[]> initParameterList() {
+
+		Object[][] parameterList = new Object[][] { 
+				{ "{\"key\":\"value\"}", 							"key" },
+				{ "{\"key\":[\"value\"]}", 							"key[0]" },
+				{ "{\"key\":[{\"key\":\"value\"}]}", 				"key[0].key" },
+				{ "{\"key\":[{\"key\":[{\"key\":\"value\"}]}]}", 	"key[0].key[0].key"},
+				{ "{\"key\":[1,[{\"key\":\"value\"}]]}", 			"key[1][0].key" },
+				{ "{\"key\":[1,[[\"key\",2,\"value\"]]]}", 			"key[1][0][2]" },
+				{ "{\"key\":{\"key\":{\"otherKey\":\"wrongValue\",\"key\":\"value\"},\"otherKey\":\"wrongValue\"},\"otherKey\":\"wrongValue\"}" , "key.key.key"}
+				};
+
+		return Arrays.asList(parameterList);
+	}
+
+	@Test
+	public void test() {
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+			String expected = "{\"retValue\":\"value\"}";
+
+			assertTrue(expected.equals(jo.toString()));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
new file mode 100644
index 0000000..6730c25
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
@@ -0,0 +1,95 @@
+/**
+ * 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.connectors.json;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.streaming.connectors.json.JSONParser;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+import org.junit.Test;
+
+
+public class JSONParserTest2 {
+	
+	@Test
+	public void testGetBooleanFunction() {
+		String jsonText = "{\"key\":true}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertTrue(jo.getBoolean("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+	@Test
+	public void testGetDoubleFunction() {
+		double expected = 12345.12345;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getDouble("retValue"),0.000001);
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+	@Test
+	public void testGetIntFunction() {
+		int expected = 15;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getInt("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+
+	@Test
+	public void testGetLongFunction() {
+		long expected = 111111111111L;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getLong("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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 cd54a54..4bb022a 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
@@ -164,17 +164,16 @@ public class JobGraphBuilder {
 	 * @param parallelism
 	 *            Number of parallel instances created
 	 */
-	public void addSource(String componentName,
-			SourceInvokable<?> InvokableObject,
+	public void addSource(String componentName, SourceInvokable<?> InvokableObject,
 			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
-		addComponent(componentName, StreamSource.class, InvokableObject,
-				operatorName, serializedFunction, parallelism);
+		addComponent(componentName, StreamSource.class, InvokableObject, operatorName,
+				serializedFunction, parallelism);
 		addTypeWrappers(componentName, null, null, outTypeWrapper, null);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SOURCE: " + componentName);
+			LOG.debug("SOURCE: {}", componentName);
 		}
 	}
 
@@ -193,11 +192,10 @@ public class JobGraphBuilder {
 	 * @param waitTime
 	 *            Max wait time for next record
 	 */
-	public void addIterationSource(String componentName, String iterationHead,
-			String iterationID, int parallelism, long waitTime) {
+	public void addIterationSource(String componentName, String iterationHead, String iterationID,
+			int parallelism, long waitTime) {
 
-		addComponent(componentName, StreamIterationSource.class, null, null,
-				null, parallelism);
+		addComponent(componentName, StreamIterationSource.class, null, null, null, parallelism);
 
 		iterationIds.put(componentName, iterationID);
 		iterationIDtoSourceName.put(iterationID, componentName);
@@ -205,14 +203,13 @@ public class JobGraphBuilder {
 		setBytesFrom(iterationHead, componentName);
 
 		setEdge(componentName, iterationHead,
-				connectionTypes.get(inEdgeList.get(iterationHead).get(0))
-						.get(0), 0, new ArrayList<String>(), false);
+				connectionTypes.get(inEdgeList.get(iterationHead).get(0)).get(0), 0,
+				new ArrayList<String>(), false);
 
-		iterationWaitTime.put(iterationIDtoSourceName.get(iterationID),
-				waitTime);
+		iterationWaitTime.put(iterationIDtoSourceName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("ITERATION SOURCE: " + componentName);
+			LOG.debug("ITERATION SOURCE: {}", componentName);
 		}
 	}
 
@@ -236,36 +233,32 @@ public class JobGraphBuilder {
 	 */
 	public <IN, OUT> void addTask(String componentName,
 			StreamOperatorInvokable<IN, OUT> taskInvokableObject,
-			TypeSerializerWrapper<?> inTypeWrapper,
-			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
-			byte[] serializedFunction, int parallelism) {
+			TypeSerializerWrapper<?> inTypeWrapper, TypeSerializerWrapper<?> outTypeWrapper,
+			String operatorName, byte[] serializedFunction, int parallelism) {
 
-		addComponent(componentName, StreamTask.class, taskInvokableObject,
-				operatorName, serializedFunction, parallelism);
+		addComponent(componentName, StreamTask.class, taskInvokableObject, operatorName,
+				serializedFunction, parallelism);
 
-		addTypeWrappers(componentName, inTypeWrapper, null, outTypeWrapper,
-				null);
+		addTypeWrappers(componentName, inTypeWrapper, null, outTypeWrapper, null);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("TASK: " + componentName);
+			LOG.debug("TASK: {}", componentName);
 		}
 	}
 
 	public <IN1, IN2, OUT> void addCoTask(String componentName,
 			CoInvokable<IN1, IN2, OUT> taskInvokableObject,
-			TypeSerializerWrapper<?> in1TypeWrapper,
-			TypeSerializerWrapper<?> in2TypeWrapper,
+			TypeSerializerWrapper<?> in1TypeWrapper, TypeSerializerWrapper<?> in2TypeWrapper,
 			TypeSerializerWrapper<?> outTypeWrapper, String operatorName,
 			byte[] serializedFunction, int parallelism) {
 
-		addComponent(componentName, CoStreamTask.class,
-				taskInvokableObject, operatorName, serializedFunction,
-				parallelism);
-		
+		addComponent(componentName, CoStreamTask.class, taskInvokableObject, operatorName,
+				serializedFunction, parallelism);
+
 		addTypeWrappers(componentName, in1TypeWrapper, in2TypeWrapper, outTypeWrapper, null);
-		
+
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("CO-TASK: " + componentName);
+			LOG.debug("CO-TASK: {}", componentName);
 		}
 	}
 
@@ -284,15 +277,15 @@ public class JobGraphBuilder {
 	 *            Number of parallel instances created
 	 */
 	public void addSink(String componentName, SinkInvokable<?> InvokableObject,
-			TypeSerializerWrapper<?> inTypeWrapper, String operatorName,
-			byte[] serializedFunction, int parallelism) {
+			TypeSerializerWrapper<?> inTypeWrapper, String operatorName, byte[] serializedFunction,
+			int parallelism) {
 
-		addComponent(componentName, StreamSink.class,
-				InvokableObject, operatorName, serializedFunction, parallelism);
+		addComponent(componentName, StreamSink.class, InvokableObject, operatorName,
+				serializedFunction, parallelism);
 		addTypeWrappers(componentName, inTypeWrapper, null, null, null);
-		
+
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK: " + componentName);
+			LOG.debug("SINK: {}", componentName);
 		}
 
 	}
@@ -315,19 +308,18 @@ public class JobGraphBuilder {
 	 * @param waitTime
 	 *            Max waiting time for next record
 	 */
-	public void addIterationSink(String componentName, String iterationTail,
-			String iterationID, int parallelism, long waitTime) {
+	public void addIterationSink(String componentName, String iterationTail, String iterationID,
+			int parallelism, long waitTime) {
 
-		addComponent(componentName, StreamIterationSink.class, null,
-				null, null, parallelism);
+		addComponent(componentName, StreamIterationSink.class, null, null, null, parallelism);
 		iterationIds.put(componentName, iterationID);
 		iterationIDtoSinkName.put(iterationID, componentName);
 		setBytesFrom(iterationTail, componentName);
-		//setInTypeWrappersFrom(iterationTail, componentName);
+		// setInTypeWrappersFrom(iterationTail, componentName);
 		iterationWaitTime.put(iterationIDtoSinkName.get(iterationID), waitTime);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("ITERATION SINK: " + componentName);
+			LOG.debug("ITERATION SINK: {}", componentName);
 		}
 
 	}
@@ -351,9 +343,8 @@ public class JobGraphBuilder {
 	 *            Number of parallel instances created
 	 */
 	private void addComponent(String componentName,
-			Class<? extends AbstractInvokable> componentClass,
-			StreamInvokable<?> invokableObject, String operatorName,
-			byte[] serializedFunction, int parallelism) {
+			Class<? extends AbstractInvokable> componentClass, StreamInvokable<?> invokableObject,
+			String operatorName, byte[] serializedFunction, int parallelism) {
 
 		componentClasses.put(componentName, componentClass);
 		setParallelism(componentName, parallelism);
@@ -366,14 +357,13 @@ public class JobGraphBuilder {
 		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<?>>());
+		connectionTypes.put(componentName, new ArrayList<StreamPartitioner<?>>());
 		iterationTailCount.put(componentName, 0);
 	}
 
-	private void addTypeWrappers(String componentName,
-			TypeSerializerWrapper<?> in1, TypeSerializerWrapper<?> in2,
-			TypeSerializerWrapper<?> out1, TypeSerializerWrapper<?> out2) {
+	private void addTypeWrappers(String componentName, TypeSerializerWrapper<?> in1,
+			TypeSerializerWrapper<?> in2, TypeSerializerWrapper<?> out1,
+			TypeSerializerWrapper<?> out2) {
 		typeWrapperIn1.put(componentName, in1);
 		typeWrapperIn2.put(componentName, in2);
 		typeWrapperOut1.put(componentName, out1);
@@ -390,10 +380,8 @@ public class JobGraphBuilder {
 	private void createVertex(String componentName) {
 
 		// Get vertex attributes
-		Class<? extends AbstractInvokable> componentClass = componentClasses
-				.get(componentName);
-		StreamInvokable<?> invokableObject = invokableObjects
-				.get(componentName);
+		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);
@@ -417,20 +405,19 @@ public class JobGraphBuilder {
 		component.setInvokableClass(componentClass);
 		component.setNumberOfSubtasks(parallelism);
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Parallelism set: " + parallelism + " for "
-					+ componentName);
+			LOG.debug("Parallelism set: {} for {}", parallelism, componentName);
 		}
 
 		StreamConfig config = new StreamConfig(component.getConfiguration());
 
 		config.setMutability(mutability.get(componentName));
 		config.setBufferTimeout(bufferTimeout.get(componentName));
-		
+
 		config.setTypeWrapperIn1(typeWrapperIn1.get(componentName));
 		config.setTypeWrapperIn2(typeWrapperIn2.get(componentName));
 		config.setTypeWrapperOut1(typeWrapperOut1.get(componentName));
 		config.setTypeWrapperOut2(typeWrapperOut2.get(componentName));
-		
+
 		// Set vertex config
 		config.setUserInvokable(invokableObject);
 		config.setComponentName(componentName);
@@ -486,10 +473,9 @@ public class JobGraphBuilder {
 	 * @param outputNames
 	 *            User defined names of the out edge
 	 */
-	public void setEdge(String upStreamComponentName,
-			String downStreamComponentName,
-			StreamPartitioner<?> partitionerObject, int typeNumber,
-			List<String> outputNames, boolean selectAll) {
+	public void setEdge(String upStreamComponentName, String downStreamComponentName,
+			StreamPartitioner<?> partitionerObject, int typeNumber, List<String> outputNames,
+			boolean selectAll) {
 		outEdgeList.get(upStreamComponentName).add(downStreamComponentName);
 		outEdgeType.get(upStreamComponentName).add(typeNumber);
 		inEdgeList.get(downStreamComponentName).add(upStreamComponentName);
@@ -509,45 +495,38 @@ public class JobGraphBuilder {
 	 * @param partitionerObject
 	 *            The partitioner
 	 */
-	private <T> void connect(String upStreamComponentName,
-			String downStreamComponentName,
+	private <T> void connect(String upStreamComponentName, String downStreamComponentName,
 			StreamPartitioner<T> partitionerObject) {
 
-		AbstractJobVertex upStreamComponent = components
-				.get(upStreamComponentName);
-		AbstractJobVertex downStreamComponent = components
-				.get(downStreamComponentName);
+		AbstractJobVertex upStreamComponent = components.get(upStreamComponentName);
+		AbstractJobVertex downStreamComponent = components.get(downStreamComponentName);
 
-		StreamConfig config = new StreamConfig(
-				upStreamComponent.getConfiguration());
+		StreamConfig config = new StreamConfig(upStreamComponent.getConfiguration());
 
 		try {
 			if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
-				upStreamComponent.connectTo(downStreamComponent,
-						ChannelType.NETWORK, DistributionPattern.POINTWISE);
+				upStreamComponent.connectTo(downStreamComponent, ChannelType.NETWORK,
+						DistributionPattern.POINTWISE);
 			} else {
-				upStreamComponent.connectTo(downStreamComponent,
-						ChannelType.NETWORK, DistributionPattern.BIPARTITE);
+				upStreamComponent.connectTo(downStreamComponent, ChannelType.NETWORK,
+						DistributionPattern.BIPARTITE);
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("CONNECTED: "
-						+ partitionerObject.getClass().getSimpleName() + " - "
-						+ upStreamComponentName + " -> "
-						+ downStreamComponentName);
+				LOG.debug("CONNECTED: {} - {} -> {}", partitionerObject.getClass().getSimpleName(),
+						upStreamComponentName, downStreamComponentName);
 			}
 
 		} catch (JobGraphDefinitionException e) {
-			throw new RuntimeException("Cannot connect components: "
-					+ upStreamComponentName + " to " + downStreamComponentName,
-					e);
+			throw new RuntimeException("Cannot connect components: " + upStreamComponentName
+					+ " to " + downStreamComponentName, e);
 		}
 
 		int outputIndex = upStreamComponent.getNumberOfForwardConnections() - 1;
 
-		config.setOutputName(outputIndex,
-				outEdgeNames.get(upStreamComponentName).get(outputIndex));
-		config.setSelectAll(outputIndex, outEdgeSelectAll.get(upStreamComponentName).get(outputIndex));
+		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));
@@ -562,12 +541,10 @@ public class JobGraphBuilder {
 	 * @param iterationTail
 	 *            ID of the iteration tail
 	 */
-	public void setIterationSourceSettings(String iterationID,
-			String iterationTail) {
+	public void setIterationSourceSettings(String iterationID, String iterationTail) {
 		setParallelism(iterationIDtoSourceName.get(iterationID),
 				componentParallelism.get(iterationTail));
-		setBufferTimeout(iterationIDtoSourceName.get(iterationID),
-				bufferTimeout.get(iterationTail));
+		setBufferTimeout(iterationIDtoSourceName.get(iterationID), bufferTimeout.get(iterationTail));
 	}
 
 	/**
@@ -580,12 +557,11 @@ public class JobGraphBuilder {
 	 * @param serializedOutputSelector
 	 *            Byte array representing the serialized output selector.
 	 */
-	public <T> void setOutputSelector(String componentName,
-			byte[] serializedOutputSelector) {
+	public <T> void setOutputSelector(String componentName, byte[] serializedOutputSelector) {
 		outputSelectors.put(componentName, serializedOutputSelector);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Outputselector set for " + componentName);
+			LOG.debug("Outputselector set for {}", componentName);
 		}
 
 	}
@@ -613,34 +589,33 @@ public class JobGraphBuilder {
 	}
 
 	public void setInToOutTypeWrappersFrom(String from, String to) {
-		//TODO rename function
+		// TODO rename function
 		typeWrapperIn1.put(to, typeWrapperOut1.get(from));
 		typeWrapperIn2.put(to, typeWrapperOut2.get(from));
 	}
-	
+
 	public void setOutToOutTypeWrappersFrom(String from, String to) {
-		//TODO rename function
+		// TODO rename function
 		typeWrapperOut1.put(to, typeWrapperOut1.get(from));
 		typeWrapperOut2.put(to, typeWrapperOut2.get(from));
 	}
-	
+
 	public void setInToInTypeWrappersFrom(String from, String to) {
-		//TODO rename function
+		// TODO rename function
 		typeWrapperIn1.put(to, typeWrapperIn1.get(from));
 		typeWrapperIn2.put(to, typeWrapperIn2.get(from));
 	}
-	
-	public TypeInformation<?> getInTypeInfo(String id){
-		// TODO 
+
+	public TypeInformation<?> getInTypeInfo(String id) {
+		// TODO
 		System.out.println("DEBUG TypeInfo " + typeWrapperIn1.get(id));
 		return typeWrapperIn1.get(id).getTypeInfo();
 	}
-	
-	public TypeInformation<?> getOutTypeInfo(String id){
-		// TODO 
+
+	public TypeInformation<?> getOutTypeInfo(String id) {
+		// TODO
 		return typeWrapperOut1.get(id).getTypeInfo();
 	}
-	
 
 	/**
 	 * Sets instance sharing between the given components
@@ -662,13 +637,11 @@ public class JobGraphBuilder {
 	 */
 	private void setAutomaticInstanceSharing() {
 
-		AbstractJobVertex maxParallelismVertex = components
-				.get(maxParallelismVertexName);
+		AbstractJobVertex maxParallelismVertex = components.get(maxParallelismVertexName);
 
 		for (String componentName : components.keySet()) {
 			if (!componentName.equals(maxParallelismVertexName)) {
-				components.get(componentName).setVertexToShareInstancesWith(
-						maxParallelismVertex);
+				components.get(componentName).setVertexToShareInstancesWith(maxParallelismVertex);
 			}
 		}
 
@@ -679,9 +652,8 @@ public class JobGraphBuilder {
 	 */
 	private void setNumberOfJobInputs() {
 		for (AbstractJobVertex component : components.values()) {
-			(new StreamConfig(component.getConfiguration()))
-					.setNumberOfInputs(component
-							.getNumberOfBackwardConnections());
+			(new StreamConfig(component.getConfiguration())).setNumberOfInputs(component
+					.getNumberOfBackwardConnections());
 		}
 	}
 
@@ -691,9 +663,8 @@ public class JobGraphBuilder {
 	 */
 	private void setNumberOfJobOutputs() {
 		for (AbstractJobVertex component : components.values()) {
-			(new StreamConfig(component.getConfiguration()))
-					.setNumberOfOutputs(component
-							.getNumberOfForwardConnections());
+			(new StreamConfig(component.getConfiguration())).setNumberOfOutputs(component
+					.getNumberOfForwardConnections());
 		}
 	}
 
@@ -710,19 +681,15 @@ public class JobGraphBuilder {
 		for (String upStreamComponentName : outEdgeList.keySet()) {
 			int i = 0;
 
-			List<Integer> outEdgeTypeList = outEdgeType
-					.get(upStreamComponentName);
+			List<Integer> outEdgeTypeList = outEdgeType.get(upStreamComponentName);
 
-			for (String downStreamComponentName : outEdgeList
-					.get(upStreamComponentName)) {
-				StreamConfig downStreamComponentConfig = new StreamConfig(
-						components.get(downStreamComponentName)
-								.getConfiguration());
+			for (String downStreamComponentName : outEdgeList.get(upStreamComponentName)) {
+				StreamConfig downStreamComponentConfig = new StreamConfig(components.get(
+						downStreamComponentName).getConfiguration());
 
 				int inputNumber = downStreamComponentConfig.getNumberOfInputs();
 
-				downStreamComponentConfig.setInputType(inputNumber++,
-						outEdgeTypeList.get(i));
+				downStreamComponentConfig.setInputType(inputNumber++, outEdgeTypeList.get(i));
 				downStreamComponentConfig.setNumberOfInputs(inputNumber);
 
 				connect(upStreamComponentName, downStreamComponentName,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index ab6caea..42a2683 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
 public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(DirectedStreamCollector.class);
-	
+
 	OutputSelector<OUT> outputSelector;
 	private List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> selectAllOutputs;
 	private Set<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> emitted;
@@ -88,9 +88,8 @@ public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
 					.get(outputName);
 			if (outputList == null) {
 				if (LOG.isErrorEnabled()) {
-					LOG.error(String.format(
-							"Cannot emit because no output is selected with the name: %s",
-							outputName));
+					LOG.error("Cannot emit because no output is selected with the name: {}",
+							outputName);
 				}
 			}
 
@@ -110,8 +109,8 @@ public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
 				}
 			} catch (Exception e) {
 				if (LOG.isErrorEnabled()) {
-					LOG.error(String.format("Emit to %s failed due to: %s", outputName,
-							StringUtils.stringifyException(e)));
+					LOG.error("Emit to {} failed due to: {}", outputName,
+							StringUtils.stringifyException(e));
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
index 54cab72..7f38df1 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/StreamCollector.java
@@ -59,7 +59,7 @@ public class StreamCollector<OUT> implements Collector<OUT> {
 	public StreamCollector(int channelID,
 			SerializationDelegate<StreamRecord<OUT>> serializationDelegate) {
 		this.serializationDelegate = serializationDelegate;
-		
+
 		if (serializationDelegate != null) {
 			this.streamRecord = serializationDelegate.getInstance();
 		} else {
@@ -104,7 +104,7 @@ public class StreamCollector<OUT> implements Collector<OUT> {
 			}
 		}
 	}
-	
+
 	/**
 	 * Collects and emits a tuple/object to the outputs by reusing a
 	 * StreamRecord object.
@@ -129,15 +129,14 @@ public class StreamCollector<OUT> implements Collector<OUT> {
 		serializationDelegate.setInstance(streamRecord);
 		emitToOutputs();
 	}
-	
+
 	protected void emitToOutputs() {
 		for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputs) {
 			try {
 				output.emit(serializationDelegate);
 			} catch (Exception e) {
 				if (LOG.isErrorEnabled()) {
-					LOG.error(String.format("Emit failed due to: %s",
-							StringUtils.stringifyException(e)));
+					LOG.error("Emit failed due to: {}", StringUtils.stringifyException(e));
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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 b2fcf89..b20e832 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
@@ -71,7 +71,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 	@Override
 	public void execute() {
 		if (LOG.isInfoEnabled()) {
-			LOG.info("Running remotely at " + host + ":" + port);
+			LOG.info("Running remotely at {}:{}", host, port);
 		}
 
 		JobGraph jobGraph = jobGraphBuilder.getJobGraph();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index 799f647..558d11c 100644
--- 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
@@ -102,8 +102,8 @@ public abstract class StreamOperatorInvokable<IN, OUT> extends StreamInvokable<O
 			callUserFunction();
 		} catch (Exception e) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error(String.format("Calling user function failed due to: %s",
-						StringUtils.stringifyException(e)));
+				LOG.error("Calling user function failed due to: {}",
+						StringUtils.stringifyException(e));
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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 c21e784..b064df7 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
@@ -17,8 +17,6 @@
 
 package org.apache.flink.streaming.api.invokable.operator.co;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.streaming.api.invokable.StreamInvokable;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
@@ -26,6 +24,8 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
 import org.apache.flink.streaming.io.CoReaderIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 
@@ -34,7 +34,7 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 	}
 
 	private static final long serialVersionUID = 1L;
-	private static final Log LOG = LogFactory.getLog(CoInvokable.class);
+	private static final Logger LOG = LoggerFactory.getLogger(CoInvokable.class);
 
 	protected CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> recordIterator;
 	protected StreamRecord<IN1> reuse1;
@@ -120,8 +120,8 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 			callUserFunction1();
 		} catch (Exception e) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error(String.format("Calling user function failed due to: %s",
-						StringUtils.stringifyException(e)));
+				LOG.error("Calling user function failed due to: {}",
+						StringUtils.stringifyException(e));
 			}
 		}
 	}
@@ -131,8 +131,8 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<OUT> {
 			callUserFunction2();
 		} catch (Exception e) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error(String.format("Calling user function failed due to: %s",
-						StringUtils.stringifyException(e)));
+				LOG.error("Calling user function failed due to: {}",
+						StringUtils.stringifyException(e));
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index 76277dc..7382d7d 100644
--- 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
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.flink.runtime.io.network.api.RecordWriter;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.api.StreamConfig;
@@ -35,9 +33,11 @@ import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
 import org.apache.flink.streaming.io.StreamRecordWriter;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
 import org.apache.flink.types.TypeInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class OutputHandler<OUT> {
-	private static final Log LOG = LogFactory.getLog(OutputHandler.class);
+	private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class);
 
 	private AbstractStreamComponent streamComponent;
 	private StreamConfig configuration;
@@ -128,14 +128,14 @@ public class OutputHandler<OUT> {
 		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: " + outputPartitioner.getClass().getSimpleName() + " with "
-					+ outputNumber + " outputs");
+			LOG.trace("Partitioner set: {} with {} outputs", outputPartitioner.getClass()
+					.getSimpleName(), outputNumber);
 		}
 	}
 
@@ -153,11 +153,11 @@ public class OutputHandler<OUT> {
 
 	long startTime;
 
-	public void invokeUserFunction(String componentTypeName,
-			StreamInvokable<OUT> userInvokable) throws IOException, InterruptedException {
+	public void invokeUserFunction(String componentTypeName, StreamInvokable<OUT> userInvokable)
+			throws IOException, InterruptedException {
 		if (LOG.isDebugEnabled()) {
-			LOG.debug(componentTypeName + " " + streamComponent.getName()
-					+ " invoked with instance id " + streamComponent.getInstanceID());
+			LOG.debug("{} {} invoked with instance id {}", componentTypeName,
+					streamComponent.getName(), streamComponent.getInstanceID());
 		}
 
 		initializeOutputSerializers();
@@ -170,8 +170,8 @@ public class OutputHandler<OUT> {
 		}
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug(componentTypeName + " " + streamComponent.getName()
-					+ " invoke finished with instance id " + streamComponent.getInstanceID());
+			LOG.debug("{} {} invoke finished instance id {}", componentTypeName,
+					streamComponent.getName(), streamComponent.getInstanceID());
 		}
 
 		flushOutputs();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index 8a1a637..40c4b96 100755
--- 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
@@ -27,13 +27,12 @@ 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 {
+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;
@@ -61,13 +60,13 @@ public class StreamIterationSink<IN extends Tuple> extends
 	@Override
 	public void invoke() throws Exception {
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK " + getName() + " invoked");
+			LOG.debug("SINK {} invoked", getName());
 		}
 
 		forwardRecords();
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK " + getName() + " invoke finished");
+			LOG.debug("SINK {} invoke finished", getName());
 		}
 	}
 
@@ -92,8 +91,8 @@ public class StreamIterationSink<IN extends Tuple> extends
 			}
 		} catch (InterruptedException e) {
 			if (LOG.isErrorEnabled()) {
-				LOG.error(String.format("Pushing back record at iteration %s failed due to: %s",
-						iterationId, StringUtils.stringifyException(e)));
+				LOG.error("Pushing back record at iteration %s failed due to: {}", iterationId,
+						StringUtils.stringifyException(e));
 			}
 			return false;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index ab02d84..3affc8c 100755
--- 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
@@ -68,7 +68,7 @@ public class StreamIterationSource<OUT extends Tuple> extends AbstractStreamComp
 	@Override
 	public void invoke() throws Exception {
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SOURCE " + getName() + " invoked with instance id " + getInstanceID());
+			LOG.debug("SOURCE {} invoked with instance id {}", getName(), getInstanceID());
 		}
 
 		outputHandler.initializeOutputSerializers();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index 0797cc1..2754c89 100644
--- 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
@@ -48,13 +48,13 @@ public class StreamSink<IN> extends AbstractStreamComponent {
 	@Override
 	public void invoke() throws Exception {
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK " + getName() + " invoked");
+			LOG.debug("SINK {} invoked", getName());
 		}
 
 		invokeUserFunction(userInvokable);
 
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("SINK " + getName() + " invoke finished");
+			LOG.debug("SINK {} invoke finished", getName());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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
index 53eafaa..52407bc 100644
--- 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
@@ -66,7 +66,7 @@ public class TestDataUtil {
 
 		if (file.exists()) {
 			if (LOG.isInfoEnabled()) {
-				LOG.info(fileName + " already exists.");
+				LOG.info("{} already exists.", fileName);
 			}
 
 			try {
@@ -93,8 +93,10 @@ public class TestDataUtil {
 	}
 
 	public static void download(String fileName) {
-		LOG.info("downloading " + fileName);
-
+		if (LOG.isInfoEnabled()) {
+			LOG.info("downloading {}", fileName);
+		}
+		
 		try {
 			URL website = new URL(testRepoUrl + fileName);
 			BufferedReader bReader = new BufferedReader(new InputStreamReader(website.openStream()));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/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 a06ceca..9c9f00d 100644
--- a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
+++ b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml
@@ -42,6 +42,12 @@ under the License.
 			<version>${project.version}</version>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-connectors</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d0dd5138/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java b/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
deleted file mode 100644
index d39996f..0000000
--- a/flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/function/JSONParseFlatMap.java
+++ /dev/null
@@ -1,144 +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.function;
-
-import org.apache.flink.api.java.functions.RichFlatMapFunction;
-import org.apache.sling.commons.json.JSONException;
-
-/**
- * Abstract class derived from {@link RichFlatMapFunction} to handle JSON files.
- * 
- * @param <IN>
- *            Type of the input elements.
- * @param <OUT>
- *            Type of the returned elements.
- */
-public abstract class JSONParseFlatMap<IN, OUT> extends RichFlatMapFunction<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	// private static final Log LOG = LogFactory.getLog(JSONParseFlatMap.class);
-
-	/**
-	 * Get the value object associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public Object get(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).get("retValue");
-	}
-
-	/**
-	 * Get the boolean value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public boolean getBoolean(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getBoolean("retValue");
-	}
-
-	/**
-	 * Get the double value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public double getDouble(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getDouble("retValue");
-	}
-
-	/**
-	 * Get the int value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public int getInt(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getInt("retValue");
-	}
-
-	/**
-	 * Get the long value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public long getLong(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getLong("retValue");
-	}
-	
-	/**
-	 * Get the String value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public String getString(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-		
-		return parser.parse(field).getString("retValue");
-	}
-}


[12/18] git commit: [streaming] Minor bug fixes in Connectors, StreamCollector & docs

Posted by mb...@apache.org.
[streaming] Minor bug fixes in Connectors, StreamCollector & docs

This closes #115


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

Branch: refs/heads/master
Commit: 439ca7ffe0a7a9fe856b22bfed50c031f062c7fb
Parents: 6b6951e
Author: Márton Balassi <ba...@gmail.com>
Authored: Sat Sep 20 13:14:57 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:44:12 2014 +0200

----------------------------------------------------------------------
 docs/streaming_guide.md                         | 81 ++++++++++----------
 .../connectors/twitter/TwitterSource.java       | 11 ++-
 .../api/collector/DirectedStreamCollector.java  | 29 ++++---
 .../api/collector/DirectedOutputTest.java       | 14 +++-
 4 files changed, 75 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/439ca7ff/docs/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming_guide.md b/docs/streaming_guide.md
index c1c6cde..87d851d 100644
--- a/docs/streaming_guide.md
+++ b/docs/streaming_guide.md
@@ -78,7 +78,7 @@ public class StreamingWordCount {
 Program Skeleton
 ----------------
 
-As we could already see in the example, a Flink Streaming program looks almost identical to a regular Flink program. Each stream processing program consists of the following parts:
+As presented in the [example](#example), a Flink Streaming program looks almost identical to a regular Flink program. Each stream processing program consists of the following parts:
 
 1. Creating a `StreamExecutionEnvironment`,
 2. Connecting to data stream sources,
@@ -100,7 +100,7 @@ For connecting to data streams the `StreamExecutionEnvironment` has many differe
 env.readTextFile(filePath)
 ```
 
-After defining the data stream sources, the user can specify transformations on the data streams to create a new data stream. Different data streams can be also combined together for joint transformations which we will see in the [operations](#operations) section.
+After defining the data stream sources, the user can specify transformations on the data streams to create a new data stream. Different data streams can be also combined together for joint transformations which are being showcased in the [operations](#operations) section.
 
 ```java
 dataStream.map(new Mapper()).reduce(new Reducer())
@@ -126,30 +126,30 @@ Basics
 
 ### DataStream
 
-The `DataStream` is the basic abstraction provided by the the Flink Streaming API. It represents a continuous stream of data of a certain type from either a data source or a transformed data stream. Operations on the DataStreams will be applied on individual data points or windows of the `DataStream` based on the type of the operation. For example the map operator transforms each data point individually while window or batch aggregations work on an interval of data points at the same time.
+The `DataStream` is the basic abstraction provided by the Flink Streaming API. It represents a continuous stream of data of a certain type from either a data source or a transformed data stream. Operations will be applied on individual data points or windows of the `DataStream` based on the type of the operation. For example the map operator transforms each data point individually while window or batch aggregations work on an interval of data points at the same time.
  
-The different operations return different `DataStream` types allowing more elaborate transformations, for example the `groupBy()` method returns a `GroupedDataStream` which can be used for group operations.
+The operations may return different `DataStream` types allowing more elaborate transformations, for example the `groupBy()` method returns a `GroupedDataStream` which can be used for group operations.
 
 ### Partitioning
 
-Partitioning controls how individual data points are distributed among the parallel instances of the transformation operators. By default Forward partitioning is used. There are several partitioning types supported in Flink Streaming:
+Partitioning controls how individual data points are distributed among the parallel instances of the transformation operators. By default *Forward* partitioning is used. There are several partitioning types supported in Flink Streaming:
 
- * Forward: Forward partitioning directs the output data to the next operator on the same core (if possible) avoiding expensive network I/O. This is the default partitioner.
+ * *Forward*: Forward partitioning directs the output data to the next operator on the same machine (if possible) avoiding expensive network I/O. This is the default partitioner.
 Usage: `dataStream.forward()`
- * Shuffle: Shuffle partitioning randomly partitions the output data stream to the next operator using uniform distribution.
+ * *Shuffle*: Shuffle partitioning randomly partitions the output data stream to the next operator using uniform distribution.
 Usage: `dataStream.shuffle()`
- * Distribute: Distribute partitioning directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution.
+ * *Distribute*: Distribute partitioning directs the output data stream to the next operator in a round-robin fashion, achieving a balanced distribution.
 Usage: `dataStream.distribute()`
- * Field: Field partitioning partitions the output data stream based on the hash code of a selected key field. Data points with the same key will always go to the same operator instance.
+ * *Field*: Field partitioning partitions the output data stream based on the hash code of a selected key field. Data points with the same key are directed to the same operator instance.
 Usage: `dataStream.partitionBy(keyposition)`
- * Broadcast: Broadcast partitioning sends the output data stream to all parallel instances of the next operator.
+ * *Broadcast*: Broadcast partitioning sends the output data stream to all parallel instances of the next operator.
 Usage: `dataStream.broadcast()`
- * Global: All data points end up at the same operator instance. To achieve a clearer structure use the parallelism setting of the corresponding operator for this.
+ * *Global*: All data points end up at the same operator instance. To achieve this use the parallelism setting of the corresponding operator.
 Usage: `operator.setParallelism(1)`
 
 ### Sources
 
-The user can connect to different data streams by the different implemenations of `DataStreamSource` using methods provided in `StreamExecutionEnvironment`. There are several predefined ones similar to the ones provided by the batch API like:
+The user can connect to data streams by the different implemenations of `DataStreamSource` using methods provided in `StreamExecutionEnvironment`. There are several predefined ones similar to the ones provided by the batch API like:
 
  * `env.genereateSequence(from, to)`
  * `env.fromElements(elements…)`
@@ -251,19 +251,19 @@ When the reduce operator is applied on a grouped data stream, the user-defined `
 
 ### Aggregations
 
-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.
+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)`
 
-For every incoming tuple the selected field is replaced with the current aggregated value. If the aggregations are used without defining field position, 0 is used as default. 
+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. 
 
 ### Window/Batch operators
 
 Window and batch operators allow the user to execute function on slices or windows of the DataStream in a sliding fashion. If the stepsize for the slide is not defined then the window/batchsize is used as stepsize by default.
 
-When applied to grouped data streams the data stream will be batched/windowed for different key values separately. 
+When applied to grouped data streams the data stream is batched/windowed for different key values separately. 
 
-For example a `ds.groupBy(0).batch(100, 10)` will produce batches of the last 100 elements for each key value with 10 record step size.
+For example a `dataStream.groupBy(0).batch(100, 10)` produces batches of the last 100 elements for each key value with 10 record step size.
  
 #### Reduce on windowed/batched data streams
 The transformation calls a user-defined `ReduceFunction` on records received in the batch or during the predefined time window. The window is shifted after each reduce call. The user can also use the different streaming aggregations.
@@ -291,10 +291,10 @@ Applies a CoMap transformation on two separate DataStreams, mapping them to a co
 A CoMap operator that outputs true if an Integer value is received and false if a String value is received:
 
 ```java
-DataStream<Integer> ds1 = ...
-DataStream<String> ds2 = ...
+DataStream<Integer> dataStream1 = ...
+DataStream<String> dataStream2 = ...
 		
-ds1.connect(ds2).
+dataStream1.connect(dataStream2)
 	.map(new CoMapFunction<Integer, String, Boolean>() {
 			
 			@Override
@@ -310,13 +310,13 @@ ds1.connect(ds2).
 ```
 
 #### FlatMap on ConnectedDataStream
-The FlatMap operator for the `ConnectedDataStream` works similarly to CoMap, but instead of returning exactly one element after each map call the user can output zero or more values using the Collector interface. 
+The FlatMap operator for the `ConnectedDataStream` works similarly to CoMap, but instead of returning exactly one element after each map call the user can output arbitrarily many values using the Collector interface. 
 
 ```java
-DataStream<Integer> ds1 = ...
-DataStream<String> ds2 = ...
-
-ds1.connect(ds2)
+DataStream<Integer> dataStream1 = ...
+DataStream<String> dataStream2 = ...
+		
+dataStream1.connect(dataStream2)
 	.flatMap(new CoFlatMapFunction<Integer, String, Boolean>() {
 
 			@Override
@@ -337,7 +337,7 @@ The Reduce operator for the `ConnectedDataStream` applies a simple reduce transf
 <section id="output-splitting">
 ### Output splitting
 
-Most data stream operators support directed outputs. It means that different data elements are received by only given outputs. The outputs are referenced by their name given at the point of receiving:
+Most data stream operators support directed outputs, meaning that different data elements are received by only given outputs. The outputs are referenced by their name given at the point of receiving:
 
 ```java
 SplitDataStream<Integer> split = someDataStream.split(outputSelector);
@@ -345,7 +345,7 @@ DataStream<Integer> even = split.select("even");
 DataStream<Integer> odd = split.select("odd");
 ```
 
-Data streams will only receive the elements directed to selected output names. These outputs are directed by implementing a selector function (extending `OutputSelector`):
+Data streams only receive the elements directed to selected output names. These outputs are directed by implementing a selector function (extending `OutputSelector`):
 
 ```java
 void select(OUT value, Collection<String> outputs);
@@ -365,7 +365,7 @@ void select(Integer value, Collection<String> outputs) {
 ```
 
 This output selection allows data streams to listen to multiple outputs, and data points to be sent to multiple outputs. A value is sent to all the outputs specified in the `OutputSelector` and a data stream will receive a value if it has selected any of the outputs the value is sent to. The stream will receive the data at most once.
-It is common that a stream needs to listen to all the outputs, so `split.selectAll()` is provided as an alias for explicitly selecting all output names.
+It is common that a stream listens to all the outputs, so `split.selectAll()` is provided as an alias for explicitly selecting all output names.
 
 
 ### Iterations
@@ -376,13 +376,13 @@ To start an iterative part of the program the user defines the iteration startin
 ```java
 IterativeDataStream<Integer> iteration = source.iterate();
 ```
-The operator applied on the iteration starting point will be the head of the iteration, where data is fed back from the iteration tail.
+The operator applied on the iteration starting point is the head of the iteration, where data is fed back from the iteration tail.
 
 ```java
 DataStream<Integer> head = iteration.map(new IterationHead());
 ```
 
-To close an iteration and define the iteration tail, the user needs to call `.closeWith(tail)` method of the `IterativeDataStream`:
+To close an iteration and define the iteration tail, the user calls `.closeWith(tail)` method of the `IterativeDataStream`:
 
 ```java
 DataStream<Integer> tail = head.map(new IterationTail());
@@ -394,19 +394,19 @@ SplitDataStream<Integer> tail = head.map(new IterationTail()).split(outputSelect
 iteration.closeWith(tail.select("iterate"));
 ``` 
 
-Because iterative streaming programs do not have a set number of iteratons for each data element, the streaming program no information on the end of its input. From this it follows that iterative streaming programs run until the user manually stops the program. While this is acceptable under normal circumstances we provide a method to allow iterative programs to shut down automatically if no input received by the iteration head for a predefined number of milliseconds.
-To use this function the user need to call, the `iteration.setMaxWaitTime(millis)` to control the max wait time. 
+Because iterative streaming programs do not have a set number of iteratons for each data element, the streaming program has no information on the end of its input. From this it follows that iterative streaming programs run until the user manually stops the program. While this is acceptable under normal circumstances a method is provided to allow iterative programs to shut down automatically if no input received by the iteration head for a predefined number of milliseconds.
+To use this function the user needs to call, the `iteration.setMaxWaitTime(millis)` to control the max wait time. 
 
 ### Rich functions
-The usage of rich functions are essentially the same as in the core Flink API. All transformations that take as argument a user-defined function can instead take as argument a rich function:
+The usage of rich functions are essentially the same as in the core Flink API. All transformations that take as argument a user-defined function can instead take a rich function as argument:
 
 ```java
-ds.map(new RichMapFunction<String, Integer>() {
+dataStream.map(new RichMapFunction<String, Integer>() {
   public Integer map(String value) { return value.toString(); }
 });
 ```
 
-Rich functions provide, in addition to the user-defined function (`map`, `reduce`, etc), the `open` and `close` methods for initialization and finalization. (In contrast to the core API, the streaming API currently does not support the  `getRuntimeContext` and `setRuntimeContext` methods.)
+Rich functions provide, in addition to the user-defined function (`map()`, `reduce()`, etc), the `open()` and `close()` methods for initialization and finalization. (In contrast to the core API, the streaming API currently does not support the  `getRuntimeContext()` and `setRuntimeContext()` methods.)
 
 [Back to top](#top)
 
@@ -416,7 +416,7 @@ Operator Settings
 
 ### Parallelism
 
-Setting parallelism for operators works exactly the same way as in the core Flink API The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(dop)` method.
+Setting parallelism for operators works exactly the same way as in the core Flink API. The user can control the number of parallel instances created for each operator by calling the `operator.setParallelism(dop)` method.
 
 ### Buffer timeout
 
@@ -432,7 +432,7 @@ env.genereateSequence(1,10).map(new MyMapper()).setBufferTimeout(timeoutMillis);
 
 ### Mutability
 
-Most operators allows setting mutability for reading input data. If the operator is set mutable then the variable used to store input data for operators will be reused in a mutable fashion to avoid excessive object creation. By default, all operators are set to immutable.
+Most operators allow setting mutability for reading input data. If the operator is set mutable then the variable used to store input data for operators will be reused in a mutable fashion to avoid excessive object creation. By default, all operators are set to immutable.
 Usage:
 ```java
 operator.setMutability(isMutable)
@@ -859,7 +859,8 @@ java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.ra
 ```
 
 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)
-```/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ```
+```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:
 
@@ -905,7 +906,8 @@ java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.ka
 ```
 
 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)
-```/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ```
+```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:
 
@@ -966,7 +968,8 @@ java -cp /PATH/TO/JAR-WITH-DEPENDENCIES org.apache.flink.streaming.connectors.fl
 ```
 
 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)
-```/git/incubator-flink/flink-addons/flink-streaming/flink-streaming-connectors/target/flink-streaming-connectors-0.7-incubating-SNAPSHOT-jar-with-dependencies.jar ```
+```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:
 
 ```

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/439ca7ff/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index 525f4c8..0ae3723 100644
--- a/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-addons/flink-streaming/flink-streaming-connectors/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -18,7 +18,6 @@
 package org.apache.flink.streaming.connectors.twitter;
 
 import java.io.FileInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.util.Properties;
 import java.util.concurrent.BlockingQueue;
@@ -27,10 +26,10 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.function.source.RichSourceFunction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.flink.streaming.api.function.source.SourceFunction;
 import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.twitter.hbc.ClientBuilder;
 import com.twitter.hbc.core.Constants;
@@ -149,8 +148,8 @@ public class TwitterSource extends RichSourceFunction<String> {
 			InputStream input = new FileInputStream(authPath);
 			properties.load(input);
 			input.close();
-		} catch (IOException ioe) {
-			new RuntimeException("Cannot open .properties file: " + authPath, ioe);
+		} catch (Exception e) {
+			throw new RuntimeException("Cannot open .properties file: " + authPath, e);
 		}
 		return properties;
 	}
@@ -226,7 +225,7 @@ public class TwitterSource extends RichSourceFunction<String> {
 				}
 			}
 		} catch (InterruptedException e) {
-			new RuntimeException("'Waiting for tweet' thread is interrupted", e);
+			throw new RuntimeException("'Waiting for tweet' thread is interrupted", e);
 		}
 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/439ca7ff/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
index 42a2683..40fe3c6 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/DirectedStreamCollector.java
@@ -86,13 +86,6 @@ public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
 		for (String outputName : outputNames) {
 			List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> outputList = outputMap
 					.get(outputName);
-			if (outputList == null) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Cannot emit because no output is selected with the name: {}",
-							outputName);
-				}
-			}
-
 			try {
 				for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : selectAllOutputs) {
 					if (!emitted.contains(output)) {
@@ -101,18 +94,32 @@ public class DirectedStreamCollector<OUT> extends StreamCollector<OUT> {
 					}
 				}
 
-				for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputList) {
-					if (!emitted.contains(output)) {
-						output.emit(serializationDelegate);
-						emitted.add(output);
+				if (outputList == null) {
+					if (LOG.isErrorEnabled()) {
+						String format = String.format(
+								"Cannot emit because no output is selected with the name: %s",
+								outputName);
+						LOG.error(format);
+
+					}
+				} else {
+
+					for (RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output : outputList) {
+						if (!emitted.contains(output)) {
+							output.emit(serializationDelegate);
+							emitted.add(output);
+						}
 					}
+
 				}
+
 			} catch (Exception e) {
 				if (LOG.isErrorEnabled()) {
 					LOG.error("Emit to {} failed due to: {}", outputName,
 							StringUtils.stringifyException(e));
 				}
 			}
+
 		}
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/439ca7ff/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index e9d3994..85f3d7f 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -43,6 +43,7 @@ public class DirectedOutputTest {
 	private static final String EVEN_AND_ODD = "evenAndOdd";
 	private static final String ODD_AND_TEN = "oddAndTen";
 	private static final String EVEN = "even";
+	private static final String NON_SELECTED = "nonSelected";
 
 	static final class MyMap implements MapFunction<Long, Long> {
 		private static final long serialVersionUID = 1L;
@@ -67,6 +68,10 @@ public class DirectedOutputTest {
 			if (value == 10L) {
 				outputs.add(TEN);
 			}
+			
+			if (value == 11L) {
+				outputs.add(NON_SELECTED);
+			}
 		}
 	}
 	
@@ -97,10 +102,11 @@ public class DirectedOutputTest {
 	
 	@Test
 	public void outputSelectorTest() throws Exception {
+		
 
 		LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 		
-		SplitDataStream<Long> source = env.generateSequence(1, 10).split(new MyOutputSelector());
+		SplitDataStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector());
 		source.select(EVEN).addSink(new ListSink(EVEN));
 		source.select(ODD, TEN).addSink(new ListSink(ODD_AND_TEN));
 		source.select(EVEN, ODD).addSink(new ListSink(EVEN_AND_ODD));
@@ -108,8 +114,8 @@ public class DirectedOutputTest {
 		
 		env.executeTest(128);
 		assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), outputs.get(EVEN));
-		assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L), outputs.get(ODD_AND_TEN));
-		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), outputs.get(EVEN_AND_ODD));
-		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L), outputs.get(ALL));
+		assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), outputs.get(ODD_AND_TEN));
+		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), outputs.get(EVEN_AND_ODD));
+		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), outputs.get(ALL));
 	}
 }


[09/18] git commit: [streaming] Invokables and tests added for fast batch/window reduce operations and aggregations

Posted by mb...@apache.org.
[streaming] Invokables and tests added for fast batch/window reduce operations and aggregations


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

Branch: refs/heads/master
Commit: 47d02a001e3450f1dcb257c7735a3cf589aa477f
Parents: 9cbd68a
Author: gyfora <gy...@gmail.com>
Authored: Sun Sep 7 21:26:35 2014 +0200
Committer: mbalassi <ba...@gmail.com>
Committed: Sat Sep 20 13:42:04 2014 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |   8 +-
 .../api/datastream/GroupedDataStream.java       |   8 +-
 .../operator/BatchGroupReduceInvokable.java     | 138 ++++++++++++----
 .../operator/BatchReduceInvokable.java          | 138 +++++-----------
 .../GroupedBatchGroupReduceInvokable.java       |  70 +++++++++
 .../operator/GroupedBatchReduceInvokable.java   |  94 +++++++++++
 .../GroupedWindowGroupReduceInvokable.java      |  70 +++++++++
 .../operator/GroupedWindowReduceInvokable.java  |  67 ++++++++
 .../operator/WindowGroupReduceInvokable.java    |  66 ++++----
 .../operator/WindowReduceInvokable.java         |  63 ++++++--
 .../streamrecord/StreamRecordSerializer.java    |   6 +-
 .../streaming/state/SlidingWindowState.java     |  12 +-
 .../operator/BatchGroupReduceTest.java          |  84 +++++-----
 .../api/invokable/operator/BatchReduceTest.java | 112 ++++++-------
 .../operator/GroupedBatchGroupReduceTest.java   |  97 ++++++++++++
 .../operator/GroupedBatchReduceTest.java        | 109 +++++++++++++
 .../WindowGroupReduceInvokableTest.java         | 117 ++++++++++++++
 .../operator/WindowReduceInvokableTest.java     | 156 +++++++++----------
 18 files changed, 1042 insertions(+), 373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/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 bd5b83c..70348d6 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
@@ -49,12 +49,12 @@ 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.operator.BatchReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FilterInvokable;
 import org.apache.flink.streaming.api.invokable.operator.FlatMapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.MapInvokable;
 import org.apache.flink.streaming.api.invokable.operator.StreamReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.WindowReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.util.DefaultTimestamp;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
 import org.apache.flink.streaming.partitioner.BroadcastPartitioner;
@@ -446,7 +446,7 @@ public class DataStream<OUT> {
 				GroupReduceFunction.class, 1);
 
 		return addFunction("batchReduce", reducer, inTypeWrapper, outTypeWrapper,
-				new BatchReduceInvokable<OUT, R>(reducer, batchSize, slideSize));
+				new BatchGroupReduceInvokable<OUT, R>(reducer, batchSize, slideSize));
 	}
 
 	/**
@@ -538,7 +538,7 @@ public class DataStream<OUT> {
 				GroupReduceFunction.class, 1);
 
 		return addFunction("batchReduce", reducer, inTypeWrapper, outTypeWrapper,
-				new WindowReduceInvokable<OUT, R>(reducer, windowSize, slideInterval, timestamp));
+				new WindowGroupReduceInvokable<OUT, R>(reducer, windowSize, slideInterval, timestamp));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/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 c5b010d..e513f2d 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
@@ -22,9 +22,9 @@ import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.functions.RichReduceFunction;
 import org.apache.flink.streaming.api.function.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.invokable.operator.BatchGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedBatchGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.operator.GroupReduceInvokable;
-import org.apache.flink.streaming.api.invokable.operator.WindowGroupReduceInvokable;
+import org.apache.flink.streaming.api.invokable.operator.GroupedWindowGroupReduceInvokable;
 import org.apache.flink.streaming.api.invokable.util.DefaultTimestamp;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
 import org.apache.flink.streaming.partitioner.StreamPartitioner;
@@ -119,7 +119,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 
 		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1), new BatchGroupReduceInvokable<OUT, R>(reducer,
+				GroupReduceFunction.class, 1), new GroupedBatchGroupReduceInvokable<OUT, R>(reducer,
 				batchSize, slideSize, keyPosition));
 	}
 
@@ -194,7 +194,7 @@ public class GroupedDataStream<OUT> extends DataStream<OUT> {
 			long windowSize, long slideInterval, Timestamp<OUT> timestamp) {
 		return addFunction("batchReduce", reducer, new FunctionTypeWrapper<OUT>(reducer,
 				GroupReduceFunction.class, 0), new FunctionTypeWrapper<R>(reducer,
-				GroupReduceFunction.class, 1), new WindowGroupReduceInvokable<OUT, R>(reducer,
+				GroupReduceFunction.class, 1), new GroupedWindowGroupReduceInvokable<OUT, R>(reducer,
 				windowSize, slideInterval, keyPosition, timestamp));
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/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 be6392e..b04cac9 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
@@ -17,54 +17,134 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.List;
 
+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.streamrecord.StreamRecord;
-import org.apache.flink.streaming.state.MutableTableState;
+import org.apache.flink.streaming.state.SlidingWindowState;
 
-public class BatchGroupReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN, OUT> {
+public class BatchGroupReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, OUT> {
 
 	private static final long serialVersionUID = 1L;
+	protected GroupReduceFunction<IN, OUT> reducer;
+	protected BatchIterator<IN> userIterator;
+	protected Iterable<IN> userIterable;
+	protected long slideSize;
+	protected long granularity;
+	protected int listSize;
+	protected transient SlidingWindowState<IN> state;
 
-	int keyPosition;
-	private Iterator<StreamRecord<IN>> iterator;
-	private MutableTableState<Object, List<IN>> values;
+	protected long batchSize;
+	protected int counter = 0;
 
 	public BatchGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long batchSize,
-			long slideSize, int keyPosition) {
-		super(reduceFunction, batchSize, slideSize);
-		this.keyPosition = keyPosition;
+			long slideSize) {
+		super(reduceFunction);
 		this.reducer = reduceFunction;
-		values = new MutableTableState<Object, List<IN>>();
+		this.batchSize = batchSize;
+		this.slideSize = slideSize;
+		this.granularity = MathUtils.gcd(batchSize, slideSize);
+		this.listSize = (int) granularity;
 	}
 
-	private IN nextValue;
+	@Override
+	protected void mutableInvoke() throws Exception {
+		throw new RuntimeException("Reducing mutable sliding batch is not supported.");
+	}
 
 	@Override
-	protected void reduce() {
-		iterator = state.getStreamRecordIterator();
-		while (iterator.hasNext()) {
-			StreamRecord<IN> nextRecord = iterator.next();
-			Object key = nextRecord.getField(keyPosition);
-			nextValue = nextRecord.getObject();
-
-			List<IN> group = values.get(key);
-			if (group != null) {
-				group.add(nextValue);
-			} else {
-				group = new ArrayList<IN>();
-				group.add(nextValue);
-				values.put(key, group);
+	protected void immutableInvoke() throws Exception {
+		if (getNextRecord() == null) {
+			throw new RuntimeException("DataStream must not be empty");
+		}
+
+		initializeAtFirstRecord();
+
+		while (reuse != null && !isStateFull()) {
+			collectOneUnit();
+		}
+		reduce();
+
+		while (reuse != null) {
+			for (int i = 0; i < slideSize / granularity; i++) {
+				if (reuse != null) {
+					collectOneUnit();
+				}
 			}
+			reduce();
+		}
+	}
+
+	protected boolean isStateFull() {
+		return state.isFull();
+	}
+
+	protected void initializeAtFirstRecord() {
+		counter = 0;
+	}
+
+	protected void collectOneUnit() throws Exception {
+		ArrayList<StreamRecord<IN>> list;
+
+		if (!batchNotFull()) {
+			list = new ArrayList<StreamRecord<IN>>();
+		} else {
+			list = new ArrayList<StreamRecord<IN>>(listSize);
+
+			do {
+				list.add(reuse);
+				resetReuse();
+			} while (getNextRecord() != null && batchNotFull());
+		}
+		state.pushBack(list);
+	}
+
+	protected StreamRecord<IN> getNextRecord() throws IOException {
+		reuse = recordIterator.next(reuse);
+		if (reuse != null) {
+			counter++;
+		}
+		return reuse;
+	}
+
+	protected boolean batchNotFull() {
+		if (counter < granularity) {
+			return true;
+		} else {
+			counter = 0;
+			return false;
 		}
-		for (List<IN> group : values.values()) {
-			userIterable = group;
-			callUserFunctionAndLogException();
+	}
+
+	protected void reduce() {
+		userIterator = state.getIterator();
+		callUserFunctionAndLogException();
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+		reducer.reduce(userIterable, collector);
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.state = new SlidingWindowState<IN>(batchSize, slideSize, granularity);
+		userIterable = new BatchIterable();
+	}
+
+	protected class BatchIterable implements Iterable<IN> {
+
+		@Override
+		public Iterator<IN> iterator() {
+			return userIterator;
 		}
-		values.clear();
+
 	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/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 0ec94ca..cfca1ab 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
@@ -17,130 +17,64 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Iterator;
 
-import org.apache.commons.math.util.MathUtils;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
+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.streamrecord.StreamRecord;
-import org.apache.flink.streaming.state.SlidingWindowState;
 
-public class BatchReduceInvokable<IN, OUT> extends StreamOperatorInvokable<IN, OUT> {
+public class BatchReduceInvokable<OUT> extends BatchGroupReduceInvokable<OUT, OUT> {
 
 	private static final long serialVersionUID = 1L;
-	protected GroupReduceFunction<IN, OUT> reducer;
-	protected BatchIterator<IN> userIterator;
-	protected Iterable<IN> userIterable;
-	protected long slideSize;
-	protected long granularity;
-	protected int listSize;
-	protected transient SlidingWindowState<IN> state;
+	protected ReduceFunction<OUT> reducer;
+	protected TypeSerializer<OUT> typeSerializer;
+	protected OUT reduceReuse;
 
-	private long batchSize;
-	private int counter = 0;
-
-	public BatchReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long batchSize,
-			long slideSize) {
-		super(reduceFunction);
+	public BatchReduceInvokable(ReduceFunction<OUT> reduceFunction, long batchSize, long slideSize) {
+		super(null, batchSize, slideSize);
 		this.reducer = reduceFunction;
-		this.batchSize = batchSize;
-		this.slideSize = slideSize;
-		this.granularity = MathUtils.gcd(batchSize, slideSize);
-		this.listSize = (int) granularity;
 	}
 
-	@Override
-	protected void mutableInvoke() throws Exception {
-		throw new RuntimeException("Reducing mutable sliding batch is not supported.");
-	}
-
-	@Override
-	protected void immutableInvoke() throws Exception {
-		if (getNextRecord() == null) {
-			throw new RuntimeException("DataStream must not be empty");
-		}
-
-		initializeAtFirstRecord();
-
-		while (reuse != null && !state.isFull()) {
-			collectOneUnit();
-		}
-		reduce();
-
-		while (reuse != null) {
-			for (int i = 0; i < slideSize / granularity; i++) {
-				if (reuse != null) {
-					collectOneUnit();
-				}
-			}
-			reduce();
-		}
-	}
-
-	protected void initializeAtFirstRecord() {
-		counter = 0;
-	}
-
-	protected void collectOneUnit() throws IOException {
-		ArrayList<StreamRecord<IN>> list;
-
-		if (!batchNotFull()) {
-			list = new ArrayList<StreamRecord<IN>>();
-		} else {
-			list = new ArrayList<StreamRecord<IN>>(listSize);
-
-			do {
-				list.add(reuse);
+	protected void collectOneUnit() throws Exception {
+		OUT reduced = null;
+		if (batchNotFull()) {
+			reduced = reuse.getObject();
+			resetReuse();
+			while (getNextRecord() != null && batchNotFull()) {
+				reduced = reducer.reduce(reduced, reuse.getObject());
 				resetReuse();
-			} while (getNextRecord() != null && batchNotFull());
-		}
-		state.pushBack(list);
-	}
-
-	protected StreamRecord<IN> getNextRecord() throws IOException {
-		reuse = recordIterator.next(reuse);
-		if (reuse != null) {
-			counter++;
-		}
-		return reuse;
-	}
-
-	protected boolean batchNotFull() {
-		if (counter < granularity) {
-			return true;
-		} else {
-			counter = 0;
-			return false;
+			}
 		}
+		state.pushBack(reduced);
 	}
 
+	@Override
 	protected void reduce() {
-		userIterator = state.getIterator();
 		callUserFunctionAndLogException();
 	}
 
 	@Override
 	protected void callUserFunction() throws Exception {
-		reducer.reduce(userIterable, collector);
+		Iterator<OUT> reducedIterator = state.getBufferIterator();
+		OUT reduced;
+		do {
+			reduced = reducedIterator.next();
+		} while (reducedIterator.hasNext() && reduced == null);
+
+		while (reducedIterator.hasNext()) {
+			OUT next = reducedIterator.next();
+			if (next != null) {
+				next = typeSerializer.copy(next, reduceReuse);
+				reduced = reducer.reduce(reduced, next);
+			}
+		}
+		collector.collect(reduced);
 	}
 
 	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.state = new SlidingWindowState<IN>(batchSize, slideSize, granularity);
-		userIterable = new BatchIterable();
-	}
-
-	protected class BatchIterable implements Iterable<IN> {
-
-		@Override
-		public Iterator<IN> iterator() {
-			return userIterator;
-		}
-
+	public void open(Configuration config) throws Exception {
+		super.open(config);
+		this.typeSerializer = serializer.getObjectSerializer();
+		this.reduceReuse = typeSerializer.createInstance();
 	}
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceInvokable.java
new file mode 100755
index 0000000..a2968a5
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceInvokable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class GroupedBatchGroupReduceInvokable<IN, OUT> extends BatchGroupReduceInvokable<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	int keyPosition;
+	private Iterator<StreamRecord<IN>> iterator;
+	private MutableTableState<Object, List<IN>> values;
+
+	public GroupedBatchGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long batchSize,
+			long slideSize, int keyPosition) {
+		super(reduceFunction, batchSize, slideSize);
+		this.keyPosition = keyPosition;
+		this.reducer = reduceFunction;
+		values = new MutableTableState<Object, List<IN>>();
+	}
+
+	private IN nextValue;
+
+	@Override
+	protected void reduce() {
+		iterator = state.getStreamRecordIterator();
+		while (iterator.hasNext()) {
+			StreamRecord<IN> nextRecord = iterator.next();
+			Object key = nextRecord.getField(keyPosition);
+			nextValue = nextRecord.getObject();
+
+			List<IN> group = values.get(key);
+			if (group != null) {
+				group.add(nextValue);
+			} else {
+				group = new ArrayList<IN>();
+				group.add(nextValue);
+				values.put(key, group);
+			}
+		}
+		for (List<IN> group : values.values()) {
+			userIterable = group;
+			callUserFunctionAndLogException();
+		}
+		values.clear();
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
new file mode 100755
index 0000000..c173932
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceInvokable.java
@@ -0,0 +1,94 @@
+/**
+ * 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 java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.state.SlidingWindowState;
+
+public class GroupedBatchReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
+
+	private static final long serialVersionUID = 1L;
+	protected transient SlidingWindowState<Map<Object, OUT>> intermediateValues;
+
+	private int keyPosition;
+
+	public GroupedBatchReduceInvokable(ReduceFunction<OUT> reduceFunction, long batchSize,
+			long slideSize, int keyPosition) {
+		super(reduceFunction, batchSize, slideSize);
+		this.keyPosition = keyPosition;
+	}
+
+	protected void collectOneUnit() throws Exception {
+		Map<Object, OUT> values = new HashMap<Object, OUT>();
+		if (batchNotFull()) {
+			do {
+				Object key = reuse.getField(keyPosition);
+				OUT nextValue = reuse.getObject();
+				OUT currentValue = values.get(key);
+				if (currentValue == null) {
+					values.put(key, nextValue);
+				} else {
+					values.put(key, reducer.reduce(currentValue, nextValue));
+				}
+				resetReuse();
+			} while (getNextRecord() != null && batchNotFull());
+		}
+		intermediateValues.pushBack(values);
+	}
+
+	@Override
+	protected boolean isStateFull() {
+		return intermediateValues.isFull();
+	}
+
+	@Override
+	protected void callUserFunction() throws Exception {
+		Iterator<Map<Object, OUT>> reducedIterator = intermediateValues.getBufferIterator();
+		Map<Object, OUT> reducedValues = reducedIterator.next();
+
+		while (reducedIterator.hasNext()) {
+			Map<Object, OUT> nextValues = reducedIterator.next();
+			for (Entry<Object, OUT> entry : nextValues.entrySet()) {
+				OUT currentValue = reducedValues.get(entry.getKey());
+				if (currentValue == null) {
+					reducedValues.put(entry.getKey(), entry.getValue());
+				} else {
+					OUT next = typeSerializer.copy(entry.getValue(), reduceReuse);
+					reducedValues.put(entry.getKey(), reducer.reduce(currentValue, next));
+				}
+			}
+		}
+		for (OUT value : reducedValues.values()) {
+			collector.collect(value);
+		}
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.intermediateValues = new SlidingWindowState<Map<Object, OUT>>(batchSize, slideSize,
+				granularity);
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
new file mode 100755
index 0000000..4027b78
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowGroupReduceInvokable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.MutableTableState;
+
+public class GroupedWindowGroupReduceInvokable<IN, OUT> extends WindowGroupReduceInvokable<IN, OUT> {
+
+	int keyPosition;
+	private Iterator<StreamRecord<IN>> iterator;
+	private MutableTableState<Object, List<IN>> values;
+
+	public GroupedWindowGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
+			long slideInterval, int keyPosition, Timestamp<IN> timestamp) {
+		super(reduceFunction, windowSize, slideInterval, timestamp);
+		this.keyPosition = keyPosition;
+		this.reducer = reduceFunction;
+		values = new MutableTableState<Object, List<IN>>();
+	}
+
+	private IN nextValue;
+
+	@Override
+	protected void reduce() {
+		iterator = state.getStreamRecordIterator();
+		while (iterator.hasNext()) {
+			StreamRecord<IN> nextRecord = iterator.next();
+			Object key = nextRecord.getField(keyPosition);
+			nextValue = nextRecord.getObject();
+
+			List<IN> group = values.get(key);
+			if (group != null) {
+				group.add(nextValue);
+			} else {
+				group = new ArrayList<IN>();
+				group.add(nextValue);
+				values.put(key, group);
+			}
+		}
+		for (List<IN> group : values.values()) {
+			userIterable = group;
+			callUserFunctionAndLogException();
+		}
+		values.clear();
+	}
+	private static final long serialVersionUID = 1L;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
new file mode 100755
index 0000000..e202e86
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowReduceInvokable.java
@@ -0,0 +1,67 @@
+/**
+ * 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 java.io.IOException;
+import java.util.Map;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.api.streamrecord.StreamRecord;
+import org.apache.flink.streaming.state.SlidingWindowState;
+
+public class GroupedWindowReduceInvokable<OUT> extends GroupedBatchReduceInvokable<OUT> {
+
+	private static final long serialVersionUID = 1L;
+	protected transient SlidingWindowState<Map<Object, OUT>> state;
+
+	private Timestamp<OUT> timestamp;
+	private long startTime;
+	private long nextRecordTime;
+
+	public GroupedWindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
+			long slideInterval, Timestamp<OUT> timestamp, int keyPosition) {
+		super(reduceFunction, windowSize, slideInterval, keyPosition);
+		this.timestamp = timestamp;
+	}
+	
+	@Override
+	protected void initializeAtFirstRecord() {
+		startTime = nextRecordTime - (nextRecordTime % granularity);
+	}
+	
+	@Override
+	protected StreamRecord<OUT> getNextRecord() throws IOException {
+		reuse = recordIterator.next(reuse);
+		if (reuse != null) {
+			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
+		}
+		return reuse;
+	}
+	
+	@Override
+	protected boolean batchNotFull() {
+		if (nextRecordTime < startTime + granularity) {
+			return true;
+		} else {
+			startTime += granularity;
+			return false;
+		}
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
index 87b00f9..7b4317a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokable.java
@@ -17,54 +17,50 @@
 
 package org.apache.flink.streaming.api.invokable.operator;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
+import java.io.IOException;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
-import org.apache.flink.streaming.state.MutableTableState;
 
-public class WindowGroupReduceInvokable<IN, OUT> extends WindowReduceInvokable<IN, OUT> {
-
-	int keyPosition;
-	private Iterator<StreamRecord<IN>> iterator;
-	private MutableTableState<Object, List<IN>> values;
+public class WindowGroupReduceInvokable<IN, OUT> extends BatchGroupReduceInvokable<IN, OUT> {
+	private static final long serialVersionUID = 1L;
+	private long startTime;
+	private long nextRecordTime;
+	private Timestamp<IN> timestamp;
 
 	public WindowGroupReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
-			long slideInterval, int keyPosition, Timestamp<IN> timestamp) {
-		super(reduceFunction, windowSize, slideInterval, timestamp);
-		this.keyPosition = keyPosition;
-		this.reducer = reduceFunction;
-		values = new MutableTableState<Object, List<IN>>();
+			long slideInterval, Timestamp<IN> timestamp) {
+		super(reduceFunction, windowSize, slideInterval);
+		this.timestamp = timestamp;
 	}
 
-	private IN nextValue;
-
 	@Override
-	protected void reduce() {
-		iterator = state.getStreamRecordIterator();
-		while (iterator.hasNext()) {
-			StreamRecord<IN> nextRecord = iterator.next();
-			Object key = nextRecord.getField(keyPosition);
-			nextValue = nextRecord.getObject();
+	protected void initializeAtFirstRecord() {
+		startTime = nextRecordTime - (nextRecordTime % granularity);
+	}
 
-			List<IN> group = values.get(key);
-			if (group != null) {
-				group.add(nextValue);
-			} else {
-				group = new ArrayList<IN>();
-				group.add(nextValue);
-				values.put(key, group);
-			}
+	@Override
+	protected StreamRecord<IN> getNextRecord() throws IOException {
+		reuse = recordIterator.next(reuse);
+		if (reuse != null) {
+			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
 		}
-		for (List<IN> group : values.values()) {
-			userIterable = group;
-			callUserFunctionAndLogException();
+		return reuse;
+	}
+	
+	@Override
+	protected boolean batchNotFull() {
+		if (nextRecordTime < startTime + granularity) {
+			return true;
+		} else {
+			startTime += granularity;
+			return false;
 		}
-		values.clear();
 	}
-	private static final long serialVersionUID = 1L;
+
+	protected void mutableInvoke() throws Exception {
+		throw new RuntimeException("Reducing mutable sliding window is not supported.");
+	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
index 49e2411..0f13397 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokable.java
@@ -18,20 +18,21 @@
 package org.apache.flink.streaming.api.invokable.operator;
 
 import java.io.IOException;
+import java.util.Iterator;
 
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
 import org.apache.flink.streaming.api.streamrecord.StreamRecord;
 
-public class WindowReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN, OUT> {
+public class WindowReduceInvokable<OUT> extends BatchReduceInvokable<OUT> {
 	private static final long serialVersionUID = 1L;
 	private long startTime;
 	private long nextRecordTime;
-	private Timestamp<IN> timestamp;
+	private Timestamp<OUT> timestamp;
+	private String nullElement = "nullElement";
 
-	public WindowReduceInvokable(GroupReduceFunction<IN, OUT> reduceFunction, long windowSize,
-			long slideInterval, Timestamp<IN> timestamp) {
+	public WindowReduceInvokable(ReduceFunction<OUT> reduceFunction, long windowSize,
+			long slideInterval, Timestamp<OUT> timestamp) {
 		super(reduceFunction, windowSize, slideInterval);
 		this.timestamp = timestamp;
 	}
@@ -41,14 +42,14 @@ public class WindowReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN, OUT
 		startTime = nextRecordTime - (nextRecordTime % granularity);
 	}
 
-	protected StreamRecord<IN> getNextRecord() throws IOException {
+	protected StreamRecord<OUT> getNextRecord() throws IOException {
 		reuse = recordIterator.next(reuse);
 		if (reuse != null) {
 			nextRecordTime = timestamp.getTimestamp(reuse.getObject());
 		}
 		return reuse;
 	}
-	
+
 	@Override
 	protected boolean batchNotFull() {
 		if (nextRecordTime < startTime + granularity) {
@@ -58,14 +59,50 @@ public class WindowReduceInvokable<IN, OUT> extends BatchReduceInvokable<IN, OUT
 			return false;
 		}
 	}
-
+	
 	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
+	protected void collectOneUnit() throws Exception {
+		OUT reduced = null;
+		if (batchNotFull()) {
+			reduced = reuse.getObject();
+			resetReuse();
+			while (getNextRecord() != null && batchNotFull()) {
+				reduced = reducer.reduce(reduced, reuse.getObject());
+				resetReuse();
+			}
+		}
+		if(reduced!=null){
+			state.pushBack(reduced);
+		}else{
+			state.pushBack(nullElement);
+		}
 	}
 
-	protected void mutableInvoke() throws Exception {
-		throw new RuntimeException("Reducing mutable sliding window is not supported.");
+	@Override
+	protected void callUserFunction() throws Exception {
+		Iterator<OUT> reducedIterator = state.getBufferIterator();
+		OUT reduced = null;
+		do {
+			OUT next = reducedIterator.next();
+			if (next != nullElement) {
+				reduced = next;
+			}
+		} while (reducedIterator.hasNext() && reduced == null);
+
+		while (reducedIterator.hasNext()) {
+			OUT next = reducedIterator.next();
+			if (next != null) {
+				try {
+					next = typeSerializer.copy(next, reduceReuse);
+					reduced = reducer.reduce(reduced, next);
+				} catch (ClassCastException e) {
+					// nullElement in buffer
+				}
+			}
+		}
+		if (reduced != null) {
+			collector.collect(reduced);
+		}
 	}
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
index 9395293..6e2fc0a 100755
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java
@@ -37,6 +37,10 @@ public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord
 		this.isTuple = typeInfo.isTupleType();
 	}
 
+	public TypeSerializer<T> getObjectSerializer() {
+		return typeSerializer;
+	}
+
 	@Override
 	public boolean isImmutableType() {
 		return false;
@@ -88,7 +92,7 @@ public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord
 
 	@Override
 	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		//Needs to be implemented
+		// Needs to be implemented
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
index 60dece2..99dba16 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/SlidingWindowState.java
@@ -52,6 +52,16 @@ public class SlidingWindowState<T> {
 		currentRecordCount += 1;
 	}
 
+	public void pushBack(Object value) {
+		buffer.add(value);
+		currentRecordCount += 1;
+	}
+
+	@SuppressWarnings("unchecked")
+	public Iterator<T> getBufferIterator() {
+		return buffer.iterator();
+	}
+
 	@SuppressWarnings("unchecked")
 	public List<StreamRecord<T>> popFront() {
 		List<StreamRecord<T>> frontRecord = (List<StreamRecord<T>>) buffer.get();
@@ -80,7 +90,7 @@ public class SlidingWindowState<T> {
 		}
 		return false;
 	}
-	
+
 	@Override
 	public String toString() {
 		return buffer.toString();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceTest.java
old mode 100755
new mode 100644
index bd6bfba..b0fa462
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchGroupReduceTest.java
@@ -19,20 +19,18 @@ package org.apache.flink.streaming.api.invokable.operator;
 
 import static org.junit.Assert.assertEquals;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.java.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.util.MockInvokable;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
 public class BatchGroupReduceTest {
 
-	public static final class MySlidingBatchReduce1 implements GroupReduceFunction<Integer, String> {
+	public static final class MySlidingBatchReduce implements GroupReduceFunction<Integer, String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -40,57 +38,61 @@ public class BatchGroupReduceTest {
 			for (Integer value : values) {
 				out.collect(value.toString());
 			}
-			out.collect(END_OF_GROUP);
+			out.collect(END_OF_BATCH);
 		}
 	}
 
-	public static final class MySlidingBatchReduce2 extends
-			RichGroupReduceFunction<Tuple2<Integer, String>, String> {
+	private final static String END_OF_BATCH = "end of batch";
+	private final static int SLIDING_BATCH_SIZE = 3;
+	private final static int SLIDE_SIZE = 2;
+
+	@Test
+	public void slidingBatchReduceTest() {
+		BatchGroupReduceInvokable<Integer, String> invokable = new BatchGroupReduceInvokable<Integer, String>(
+				new MySlidingBatchReduce(), SLIDING_BATCH_SIZE, SLIDE_SIZE);
+
+		List<String> expected = Arrays.asList("1", "2", "3", END_OF_BATCH, "3", "4", "5",
+				END_OF_BATCH, "5", "6", "7", END_OF_BATCH);
+		List<String> actual = MockInvokable.createAndExecute(invokable,
+				Arrays.asList(1, 2, 3, 4, 5, 6, 7));
+
+		assertEquals(expected, actual);
+	}
+
+	public static final class MyBatchReduce implements GroupReduceFunction<Double, Double> {
 		private static final long serialVersionUID = 1L;
 
-		String openString;
-		
 		@Override
-		public void reduce(Iterable<Tuple2<Integer, String>> values, Collector<String> out)
-				throws Exception {
-			out.collect(openString);
-			for (Tuple2<Integer, String> value : values) {
-				out.collect(value.f0.toString());
+		public void reduce(Iterable<Double> values, Collector<Double> out) throws Exception {
+
+			Double sum = 0.;
+			Double count = 0.;
+			for (Double value : values) {
+				sum += value;
+				count++;
+			}
+			if (count > 0) {
+				out.collect(new Double(sum / count));
 			}
-			out.collect(END_OF_GROUP);
-		}
-		
-		@Override
-		public void open(Configuration c){
-			openString = "open";
 		}
 	}
 
-	private final static String END_OF_GROUP = "end of group";
+	private static final int BATCH_SIZE = 5;
 
-	@SuppressWarnings("unchecked")
 	@Test
-	public void slidingBatchGroupReduceTest() {
-		BatchGroupReduceInvokable<Integer, String> invokable1 = new BatchGroupReduceInvokable<Integer, String>(
-				new MySlidingBatchReduce1(), 3, 2, 0);
-
-		List<String> expected = Arrays.asList("1", "1", END_OF_GROUP, "2", END_OF_GROUP, "2",
-				END_OF_GROUP, "3", "3", END_OF_GROUP);
-		List<String> actual = MockInvokable.createAndExecute(invokable1,
-				Arrays.asList(1, 1, 2, 3, 3));
-
-		assertEquals(expected, actual);
-
-		BatchGroupReduceInvokable<Tuple2<Integer, String>, String> invokable2 = new BatchGroupReduceInvokable<Tuple2<Integer, String>, String>(
-				new MySlidingBatchReduce2(), 2, 2, 1);
+	public void nonSlidingBatchReduceTest() {
+		List<Double> inputs = new ArrayList<Double>();
+		for (Double i = 1.; i <= 100; i++) {
+			inputs.add(i);
+		}
 
-		expected = Arrays.asList("open","1", "2", END_OF_GROUP,"open", "3", END_OF_GROUP,"open", "4", END_OF_GROUP);
-		actual = MockInvokable.createAndExecute(invokable2, Arrays.asList(
-				new Tuple2<Integer, String>(1, "a"), new Tuple2<Integer, String>(2, "a"),
-				new Tuple2<Integer, String>(3, "b"), new Tuple2<Integer, String>(4, "a")));
+		BatchGroupReduceInvokable<Double, Double> invokable = new BatchGroupReduceInvokable<Double, Double>(
+				new MyBatchReduce(), BATCH_SIZE, BATCH_SIZE);
 
-		assertEquals(expected, actual);
+		List<Double> avgs = MockInvokable.createAndExecute(invokable, inputs);
 
+		for (int i = 0; i < avgs.size(); i++) {
+			assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0);
+		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
old mode 100644
new mode 100755
index 85534ad..3140dc0
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceTest.java
@@ -20,79 +20,67 @@ package org.apache.flink.streaming.api.invokable.operator;
 import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
-import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.streaming.util.MockInvokable;
-import org.apache.flink.util.Collector;
 import org.junit.Test;
 
-
 public class BatchReduceTest {
 
-	public static final class MySlidingBatchReduce implements GroupReduceFunction<Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Integer> values, Collector<String> out) throws Exception {
-			for (Integer value : values) {
-				out.collect(value.toString());
-			}
-			out.collect(END_OF_BATCH);
-		}
-	}
-
-	private final static String END_OF_BATCH = "end of batch";
-	private final static int SLIDING_BATCH_SIZE = 3;
-	private final static int SLIDE_SIZE = 2;
-
 	@Test
-	public void slidingBatchReduceTest() {
-		BatchReduceInvokable<Integer, String> invokable = new BatchReduceInvokable<Integer, String>(
-				new MySlidingBatchReduce(), SLIDING_BATCH_SIZE, SLIDE_SIZE);
+	public void BatchReduceInvokableTest() {
 
-		List<String> expected = Arrays.asList("1", "2", "3", END_OF_BATCH, "3", "4", "5",
-				END_OF_BATCH, "5", "6", "7", END_OF_BATCH);
-		List<String> actual = MockInvokable.createAndExecute(invokable,
-				Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-
-		assertEquals(expected, actual);
-	}
-
-	public static final class MyBatchReduce implements GroupReduceFunction<Double, Double> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Double> values, Collector<Double> out) throws Exception {
-
-			Double sum = 0.;
-			Double count = 0.;
-			for (Double value : values) {
-				sum += value;
-				count++;
-			}
-			if (count > 0) {
-				out.collect(new Double(sum / count));
-			}
-		}
-	}
-	
-	private static final int BATCH_SIZE = 5;
-
-	@Test
-	public void nonSlidingBatchReduceTest() {
-		List<Double> inputs = new ArrayList<Double>();
-		for (Double i = 1.; i <= 100; i++) {
+		List<Integer> inputs = new ArrayList<Integer>();
+		for (Integer i = 1; i <= 10; i++) {
 			inputs.add(i);
 		}
-		
-		BatchReduceInvokable<Double, Double> invokable = new BatchReduceInvokable<Double, Double>(new MyBatchReduce(), BATCH_SIZE, BATCH_SIZE);
-		
-		List<Double> avgs = MockInvokable.createAndExecute(invokable, inputs);
+		BatchReduceInvokable<Integer> invokable = new BatchReduceInvokable<Integer>(
+				new ReduceFunction<Integer>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Integer reduce(Integer value1, Integer value2) throws Exception {
+						return value1 + value2;
+					}
+				}, 3, 2);
+
+		List<Integer> expected = new ArrayList<Integer>();
+		expected.add(6);
+		expected.add(12);
+		expected.add(18);
+		expected.add(24);
+		expected.add(27);
+		assertEquals(expected, MockInvokable.createAndExecute(invokable, inputs));
+
+		List<Integer> inputs2 = new ArrayList<Integer>();
+		inputs2.add(1);
+		inputs2.add(2);
+		inputs2.add(-1);
+		inputs2.add(-3);
+		inputs2.add(3);
+
+		BatchReduceInvokable<Integer> invokable2 = new BatchReduceInvokable<Integer>(
+				new ReduceFunction<Integer>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Integer reduce(Integer value1, Integer value2) throws Exception {
+						if (value1 <= value2) {
+							return value1;
+						} else {
+							return value2;
+						}
+					}
+				}, 2, 2);
+
+		List<Integer> expected2 = new ArrayList<Integer>();
+		expected2.add(1);
+		expected2.add(-3);
+		expected2.add(3);
+
+		assertEquals(expected2, MockInvokable.createAndExecute(invokable2, inputs2));
 
-		for (int i = 0; i < avgs.size(); i++) {
-			assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0);
-		}
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
new file mode 100755
index 0000000..096141d
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchGroupReduceTest.java
@@ -0,0 +1,97 @@
+/**
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.java.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+public class GroupedBatchGroupReduceTest {
+
+	public static final class MySlidingBatchReduce1 implements GroupReduceFunction<Integer, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Integer> values, Collector<String> out) throws Exception {
+			for (Integer value : values) {
+				out.collect(value.toString());
+			}
+			out.collect(END_OF_GROUP);
+		}
+	}
+
+	public static final class MySlidingBatchReduce2 extends
+			RichGroupReduceFunction<Tuple2<Integer, String>, String> {
+		private static final long serialVersionUID = 1L;
+
+		String openString;
+
+		@Override
+		public void reduce(Iterable<Tuple2<Integer, String>> values, Collector<String> out)
+				throws Exception {
+			out.collect(openString);
+			for (Tuple2<Integer, String> value : values) {
+				out.collect(value.f0.toString());
+			}
+			out.collect(END_OF_GROUP);
+		}
+
+		@Override
+		public void open(Configuration c) {
+			openString = "open";
+		}
+	}
+
+	private final static String END_OF_GROUP = "end of group";
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void slidingBatchGroupReduceTest() {
+		GroupedBatchGroupReduceInvokable<Integer, String> invokable1 = new GroupedBatchGroupReduceInvokable<Integer, String>(
+				new MySlidingBatchReduce1(), 3, 2, 0);
+
+		List<String> expected = Arrays.asList("1", "1", END_OF_GROUP, "2", END_OF_GROUP, "2",
+				END_OF_GROUP, "3", "3", END_OF_GROUP);
+		List<String> actual = MockInvokable.createAndExecute(invokable1,
+				Arrays.asList(1, 1, 2, 3, 3));
+
+		assertEquals(expected, actual);
+
+		GroupedBatchGroupReduceInvokable<Tuple2<Integer, String>, String> invokable2 = new GroupedBatchGroupReduceInvokable<Tuple2<Integer, String>, String>(
+				new MySlidingBatchReduce2(), 2, 2, 1);
+
+		expected = Arrays.asList("open", "1", "2", END_OF_GROUP, "open", "3", END_OF_GROUP, "open",
+				"4", END_OF_GROUP);
+		actual = MockInvokable.createAndExecute(invokable2, Arrays.asList(
+				new Tuple2<Integer, String>(1, "a"), new Tuple2<Integer, String>(2, "a"),
+				new Tuple2<Integer, String>(3, "b"), new Tuple2<Integer, String>(4, "a")));
+
+		assertEquals(expected, actual);
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.java
new file mode 100755
index 0000000..850ac8d
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/GroupedBatchReduceTest.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.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.junit.Test;
+
+public class GroupedBatchReduceTest {
+
+	@Test
+	public void BatchReduceInvokableTest() {
+
+		List<Integer> inputs = new ArrayList<Integer>();
+		inputs.add(1);
+		inputs.add(1);
+		inputs.add(5);
+		inputs.add(5);
+		inputs.add(5);
+		inputs.add(1);
+		inputs.add(1);
+		inputs.add(5);
+		inputs.add(1);
+		inputs.add(5);
+
+		GroupedBatchReduceInvokable<Integer> invokable = new GroupedBatchReduceInvokable<Integer>(
+				new ReduceFunction<Integer>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Integer reduce(Integer value1, Integer value2) throws Exception {
+						return value1 + value2;
+					}
+				}, 4, 2, 0);
+
+		List<Integer> expected = new ArrayList<Integer>();
+		expected.add(2);
+		expected.add(10);
+		expected.add(1);
+		expected.add(15);
+		expected.add(2);
+		expected.add(10);
+		expected.add(2);
+		expected.add(10);
+		List<Integer> actual = MockInvokable.createAndExecute(invokable, inputs);
+		assertEquals(new HashSet<Integer>(expected), new HashSet<Integer>(actual));
+		assertEquals(expected.size(), actual.size());
+
+		List<Tuple2<Integer, String>> inputs2 = new ArrayList<Tuple2<Integer, String>>();
+		inputs2.add(new Tuple2<Integer, String>(1, "a"));
+		inputs2.add(new Tuple2<Integer, String>(0, "b"));
+		inputs2.add(new Tuple2<Integer, String>(2, "a"));
+		inputs2.add(new Tuple2<Integer, String>(-1, "a"));
+		inputs2.add(new Tuple2<Integer, String>(-2, "a"));
+		inputs2.add(new Tuple2<Integer, String>(10, "a"));
+		inputs2.add(new Tuple2<Integer, String>(2, "b"));
+		inputs2.add(new Tuple2<Integer, String>(1, "a"));
+
+		GroupedBatchReduceInvokable<Tuple2<Integer, String>> invokable2 = new GroupedBatchReduceInvokable<Tuple2<Integer, String>>(
+				new ReduceFunction<Tuple2<Integer, String>>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Tuple2<Integer, String> reduce(Tuple2<Integer, String> value1,
+							Tuple2<Integer, String> value2) throws Exception {
+						if (value1.f0 <= value2.f0) {
+							return value1;
+						} else {
+							return value2;
+						}
+					}
+				}, 3, 3, 1);
+
+		List<Tuple2<Integer, String>> expected2 = new ArrayList<Tuple2<Integer, String>>();
+		expected2.add(new Tuple2<Integer, String>(1, "a"));
+		expected2.add(new Tuple2<Integer, String>(0, "b"));
+		expected2.add(new Tuple2<Integer, String>(-2, "a"));
+		expected2.add(new Tuple2<Integer, String>(2, "b"));
+		expected2.add(new Tuple2<Integer, String>(1, "a"));
+
+		List<Tuple2<Integer, String>> actual2 = MockInvokable.createAndExecute(invokable2, inputs2);
+		assertEquals(new HashSet<Tuple2<Integer, String>>(expected2),
+				new HashSet<Tuple2<Integer, String>>(actual2));
+		assertEquals(expected2.size(), actual2.size());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
new file mode 100644
index 0000000..7437bec
--- /dev/null
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowGroupReduceInvokableTest.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.invokable.operator;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.streaming.api.invokable.util.Timestamp;
+import org.apache.flink.streaming.util.MockInvokable;
+import org.apache.flink.util.Collector;
+import org.junit.Before;
+import org.junit.Test;
+
+public class WindowGroupReduceInvokableTest {
+
+	public static final class MySlidingWindowReduce implements GroupReduceFunction<Integer, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Integer> values, Collector<String> out) throws Exception {
+			for (Integer value : values) {
+				out.collect(value.toString());
+			}
+			out.collect(EOW);
+		}
+	}
+
+	public static final class MyTimestamp implements Timestamp<Integer> {
+		private static final long serialVersionUID = 1L;
+
+		private Iterator<Long> timestamps;
+
+		public MyTimestamp(List<Long> timestamps) {
+			this.timestamps = timestamps.iterator();
+		}
+
+		@Override
+		public long getTimestamp(Integer value) {
+			long ts = timestamps.next();
+			return ts;
+		}
+	}
+
+	private final static String EOW = "|";
+
+	private static List<WindowGroupReduceInvokable<Integer, String>> invokables = new ArrayList<WindowGroupReduceInvokable<Integer, String>>();
+	private static List<List<String>> expectedResults = new ArrayList<List<String>>();
+
+	@Before
+	public void before() {
+		long windowSize = 3;
+		long slideSize = 2;
+		List<Long> timestamps = Arrays.asList(101L, 102L, 103L, 104L, 105L, 106L, 107L, 108L, 109L,
+				110L);
+		expectedResults.add(Arrays.asList("1", "2", "3", EOW, "3", "4", "5", EOW, "5", "6", "7",
+				EOW, "7", "8", "9", EOW, "8", "9", "10", EOW));
+		invokables.add(new WindowGroupReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
+				windowSize, slideSize, new MyTimestamp(timestamps)));
+
+		windowSize = 10;
+		slideSize = 5;
+		timestamps = Arrays.asList(101L, 103L, 121L, 122L, 123L, 124L, 180L, 181L, 185L, 190L);
+		expectedResults.add(Arrays.asList("1", "2", EOW, EOW, EOW, "3", "4", "5", "6", EOW, "3",
+				"4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", "8",
+				EOW, "7", "8", "9", EOW, "9", "10", EOW));
+		invokables.add(new WindowGroupReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
+				windowSize, slideSize, new MyTimestamp(timestamps)));
+
+		windowSize = 10;
+		slideSize = 4;
+		timestamps = Arrays.asList(101L, 103L, 110L, 112L, 113L, 114L, 120L, 121L, 125L, 130L);
+		expectedResults.add(Arrays.asList("1", "2", EOW, "3", "4", "5", EOW, "3", "4", "5", "6",
+				EOW, "4", "5", "6", "7", "8", EOW, "7", "8", "9", EOW, "7", "8", "9", EOW, "9",
+				"10", EOW));
+		invokables.add(new WindowGroupReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
+				windowSize, slideSize, new MyTimestamp(timestamps)));
+	}
+
+	@Test
+	public void slidingBatchReduceTest() {
+		List<List<String>> actualResults = new ArrayList<List<String>>();
+
+		for (WindowGroupReduceInvokable<Integer, String> invokable : invokables) {
+			List<String> result = MockInvokable.createAndExecute(invokable,
+					Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
+			actualResults.add(result);
+		}
+
+		Iterator<List<String>> actualResult = actualResults.iterator();
+
+		for (List<String> expectedResult : expectedResults) {
+			assertEquals(expectedResult, actualResult.next());
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/47d02a00/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
old mode 100644
new mode 100755
index 6dec400..1aed25f
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/WindowReduceInvokableTest.java
@@ -20,97 +20,91 @@ package org.apache.flink.streaming.api.invokable.operator;
 import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
+import java.util.HashSet;
 import java.util.List;
 
-import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.invokable.util.Timestamp;
 import org.apache.flink.streaming.util.MockInvokable;
-import org.apache.flink.util.Collector;
-import org.junit.Before;
 import org.junit.Test;
 
 public class WindowReduceInvokableTest {
 
-	public static final class MySlidingWindowReduce implements GroupReduceFunction<Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Integer> values, Collector<String> out) throws Exception {
-			for (Integer value : values) {
-				out.collect(value.toString());
-			}
-			out.collect(EOW);
-		}
-	}
-
-	public static final class MyTimestamp implements Timestamp<Integer> {
-		private static final long serialVersionUID = 1L;
-
-		private Iterator<Long> timestamps;
-
-		public MyTimestamp(List<Long> timestamps) {
-			this.timestamps = timestamps.iterator();
-		}
-
-		@Override
-		public long getTimestamp(Integer value) {
-			long ts = timestamps.next();
-			return ts;
-		}
-	}
-
-	private final static String EOW = "|";
-
-	private static List<WindowReduceInvokable<Integer, String>> invokables = new ArrayList<WindowReduceInvokable<Integer, String>>();
-	private static List<List<String>> expectedResults = new ArrayList<List<String>>();
-
-	@Before
-	public void before() {
-		long windowSize = 3;
-		long slideSize = 2;
-		List<Long> timestamps = Arrays.asList(101L, 102L, 103L, 104L, 105L, 106L, 107L, 108L, 109L,
-				110L);
-		expectedResults.add(Arrays.asList("1", "2", "3", EOW, "3", "4", "5", EOW, "5", "6", "7",
-				EOW, "7", "8", "9", EOW, "8", "9", "10", EOW));
-		invokables.add(new WindowReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
-				windowSize, slideSize, new MyTimestamp(timestamps)));
-
-		windowSize = 10;
-		slideSize = 5;
-		timestamps = Arrays.asList(101L, 103L, 121L, 122L, 123L, 124L, 180L, 181L, 185L, 190L);
-		expectedResults.add(Arrays.asList("1", "2", EOW, EOW, EOW, "3", "4", "5", "6", EOW, "3",
-				"4", "5", "6", EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, EOW, "7", "8",
-				EOW, "7", "8", "9", EOW, "9", "10", EOW));
-		invokables.add(new WindowReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
-				windowSize, slideSize, new MyTimestamp(timestamps)));
-
-		windowSize = 10;
-		slideSize = 4;
-		timestamps = Arrays.asList(101L, 103L, 110L, 112L, 113L, 114L, 120L, 121L, 125L, 130L);
-		expectedResults.add(Arrays.asList("1", "2", EOW, "3", "4", "5", EOW, "3", "4", "5", "6",
-				EOW, "4", "5", "6", "7", "8", EOW, "7", "8", "9", EOW, "7", "8", "9", EOW, "9",
-				"10", EOW));
-		invokables.add(new WindowReduceInvokable<Integer, String>(new MySlidingWindowReduce(),
-				windowSize, slideSize, new MyTimestamp(timestamps)));
-	}
-
 	@Test
-	public void slidingBatchReduceTest() {
-		List<List<String>> actualResults = new ArrayList<List<String>>();
-
-		for (WindowReduceInvokable<Integer, String> invokable : invokables) {
-			List<String> result = MockInvokable.createAndExecute(invokable,
-					Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10));
-			actualResults.add(result);
-		}
-
-		Iterator<List<String>> actualResult = actualResults.iterator();
+	public void windowReduceTest() {
+
+		List<Integer> inputs = new ArrayList<Integer>();
+		inputs.add(1);
+		inputs.add(2);
+		inputs.add(2);
+		inputs.add(3);
+		inputs.add(4);
+		inputs.add(5);
+		inputs.add(10);
+		inputs.add(11);
+		inputs.add(11);
+		WindowReduceInvokable<Integer> invokable = new WindowReduceInvokable<Integer>(
+				new ReduceFunction<Integer>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Integer reduce(Integer value1, Integer value2) throws Exception {
+						return value1 + value2;
+					}
+				}, 4, 2, new Timestamp<Integer>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public long getTimestamp(Integer value) {
+						return value;
+					}
+				});
+
+		List<Integer> expected = new ArrayList<Integer>();
+		expected.add(8);
+		expected.add(16);
+		expected.add(9);
+		expected.add(32);
+		assertEquals(expected, MockInvokable.createAndExecute(invokable, inputs));
+
+		List<Tuple2<String, Integer>> inputs2 = new ArrayList<Tuple2<String, Integer>>();
+		inputs2.add(new Tuple2<String, Integer>("a", 1));
+		inputs2.add(new Tuple2<String, Integer>("a", 2));
+		inputs2.add(new Tuple2<String, Integer>("b", 2));
+		inputs2.add(new Tuple2<String, Integer>("b", 2));
+		inputs2.add(new Tuple2<String, Integer>("a", 3));
+		inputs2.add(new Tuple2<String, Integer>("b", 4));
+		inputs2.add(new Tuple2<String, Integer>("b", 5));
+
+		GroupedWindowReduceInvokable<Tuple2<String, Integer>> invokable2 = new GroupedWindowReduceInvokable<Tuple2<String, Integer>>(
+				new ReduceFunction<Tuple2<String, Integer>>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
+							Tuple2<String, Integer> value2) throws Exception {
+						return new Tuple2<String, Integer>(value1.f0, value1.f1 + value2.f1);
+					}
+				}, 3, 2, new Timestamp<Tuple2<String, Integer>>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public long getTimestamp(Tuple2<String, Integer> value) {
+						return value.f1;
+					}
+				}, 0);
+
+		List<Tuple2<String, Integer>> expected2 = new ArrayList<Tuple2<String, Integer>>();
+		expected2.add(new Tuple2<String, Integer>("a", 6));
+		expected2.add(new Tuple2<String, Integer>("b", 4));
+		expected2.add(new Tuple2<String, Integer>("b", 9));
+		expected2.add(new Tuple2<String, Integer>("a", 3));
+		List<Tuple2<String, Integer>> actual2 = MockInvokable.createAndExecute(invokable2, inputs2);
+		assertEquals(new HashSet<Tuple2<String, Integer>>(expected2),
+				new HashSet<Tuple2<String, Integer>>(actual2));
+		assertEquals(expected2.size(), actual2.size());
 
-		for (List<String> expectedResult : expectedResults) {
-			assertEquals(expectedResult, actualResult.next());
-		}
 	}
 
 }