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

[01/13] flink git commit: [FLINK-2550] Rename SplitDataStream to SplitStream

Repository: flink
Updated Branches:
  refs/heads/master 68c1afc20 -> 9513f0e33


[FLINK-2550] Rename SplitDataStream to SplitStream


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

Branch: refs/heads/master
Commit: 9513f0e33f7aba46ebcc322d51ef12f0302ec2c2
Parents: 7b6e762
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 5 14:20:33 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../api/FlinkTopologyBuilder.java               |  7 ++-
 .../util/SplitStreamMapper.java                 |  4 +-
 .../split/SpoutSplitExample.java                |  4 +-
 .../api/collector/selector/OutputSelector.java  |  6 +-
 .../streaming/api/datastream/DataStream.java    |  8 +--
 .../api/datastream/SplitDataStream.java         | 62 --------------------
 .../streaming/api/datastream/SplitStream.java   | 62 ++++++++++++++++++++
 .../flink/streaming/api/DataStreamTest.java     |  4 +-
 .../apache/flink/streaming/api/IterateTest.java |  6 +-
 .../streaming/api/StreamingOperatorsITCase.java |  4 +-
 .../api/collector/DirectedOutputTest.java       |  4 +-
 .../api/complex/ComplexIntegrationTest.java     |  4 +-
 .../examples/iteration/IterateExample.java      |  4 +-
 .../flink/streaming/api/scala/DataStream.scala  |  8 +--
 .../streaming/api/scala/SplitDataStream.scala   | 37 ------------
 .../flink/streaming/api/scala/SplitStream.scala | 37 ++++++++++++
 .../flink/streaming/api/scala/package.scala     |  6 +-
 .../StreamingScalaAPICompletenessTest.scala     |  6 +-
 18 files changed, 137 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
index e2d819c..e4f6c94 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
@@ -1,4 +1,5 @@
 /*
+/*
  * 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
@@ -44,7 +45,7 @@ import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 
 import java.util.HashMap;
 import java.util.HashSet;
@@ -112,7 +113,7 @@ public class FlinkTopologyBuilder {
 			} else {
 				source = env.addSource(spoutWrapper, spoutId,
 						TypeExtractor.getForClass(SplitStreamType.class));
-				SplitDataStream splitSource = source.split(new FlinkStormStreamSelector());
+				SplitStream splitSource = source.split(new FlinkStormStreamSelector());
 
 				for (String streamId : sourceStreams.keySet()) {
 					outputStreams.put(streamId, splitSource.select(streamId));
@@ -246,7 +247,7 @@ public class FlinkTopologyBuilder {
 										new StormBoltWrapper(userBolt, this.outputStreams.get(
 												producerId).get(inputStreamId)));
 
-								SplitDataStream splitStreams = outputStream
+								SplitStream splitStreams = outputStream
 										.split(new FlinkStormStreamSelector());
 
 								HashMap<String, DataStream> op = new HashMap<String, DataStream>();

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
index afcdcae..9cb44ec 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
@@ -18,13 +18,13 @@ package org.apache.flink.stormcompatibility.util;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 
 /**
  * Strips {@link SplitStreamType}{@code <T>} away, ie, extracts the wrapped record of type {@code T}. Can be used to get
  * a "clean" stream from a Spout/Bolt that declared multiple output streams (after the streams got separated using
  * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} and
- * {@link SplitDataStream#select(String...) .select(...)}).
+ * {@link SplitStream#select(String...) .select(...)}).
  * 
  * @param <T>
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
index 4116f3c..18251d4 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
@@ -28,7 +28,7 @@ import org.apache.flink.stormcompatibility.util.SplitStreamType;
 import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper;
 import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 /**
@@ -60,7 +60,7 @@ public class SpoutSplitExample {
 				new StormSpoutWrapper<SplitStreamType<Integer>>(new RandomSpout(true, 0),
 						rawOutputs), TypeExtractor.getForObject(new SplitStreamType<Integer>()));
 
-		SplitDataStream<SplitStreamType<Integer>> splitStream = numbers
+		SplitStream<SplitStreamType<Integer>> splitStream = numbers
 				.split(new FlinkStormStreamSelector<Integer>());
 
 		DataStream<SplitStreamType<Integer>> evenStream = splitStream.select(RandomSpout.EVEN_STREAM);

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
index b886fa6..9c6eede 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
@@ -20,12 +20,12 @@ package org.apache.flink.streaming.api.collector.selector;
 import java.io.Serializable;
 
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 
 /**
- * Interface for defining an OutputSelector for a {@link SplitDataStream} using
+ * Interface for defining an OutputSelector for a {@link SplitStream} using
  * the {@link SingleOutputStreamOperator#split} call. Every output object of a
- * {@link SplitDataStream} will run through this operator to select outputs.
+ * {@link SplitStream} will run through this operator to select outputs.
  * 
  * @param <OUT>
  *            Type parameter of the split values.

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 003ef36..8de1a0d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -204,16 +204,16 @@ public class DataStream<T> {
 	/**
 	 * Operator used for directing tuples to specific named outputs using an
 	 * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}.
-	 * Calling this method on an operator creates a new {@link SplitDataStream}.
+	 * Calling this method on an operator creates a new {@link SplitStream}.
 	 * 
 	 * @param outputSelector
 	 *            The user defined
 	 *            {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}
 	 *            for directing the tuples.
-	 * @return The {@link SplitDataStream}
+	 * @return The {@link SplitStream}
 	 */
-	public SplitDataStream<T> split(OutputSelector<T> outputSelector) {
-		return new SplitDataStream<T>(this, clean(outputSelector));
+	public SplitStream<T> split(OutputSelector<T> outputSelector) {
+		return new SplitStream<T>(this, clean(outputSelector));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
deleted file mode 100644
index bc9ecfb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitDataStream.java
+++ /dev/null
@@ -1,62 +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 com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.transformations.SelectTransformation;
-import org.apache.flink.streaming.api.transformations.SplitTransformation;
-
-/**
- * The SplitDataStream represents an operator that has been split using an
- * {@link OutputSelector}. Named outputs can be selected using the
- * {@link #select} function. To apply transformation on the whole output simply
- * call the transformation on the SplitDataStream
- *
- * @param <OUT> The type of the elements in the Stream
- */
-public class SplitDataStream<OUT> extends DataStream<OUT> {
-
-	protected SplitDataStream(DataStream<OUT> dataStream, OutputSelector<OUT> outputSelector) {
-		super(dataStream.getExecutionEnvironment(), new SplitTransformation<OUT>(dataStream.getTransformation(), outputSelector));
-	}
-
-	/**
-	 * Sets the output names for which the next operator will receive values.
-	 * 
-	 * @param outputNames
-	 *            The output names for which the operator will receive the
-	 *            input.
-	 * @return Returns the selected DataStream
-	 */
-	public DataStream<OUT> select(String... outputNames) {
-		return selectOutput(outputNames);
-	}
-
-	private DataStream<OUT> selectOutput(String[] outputNames) {
-		for (String outName : outputNames) {
-			if (outName == null) {
-				throw new RuntimeException("Selected names must not be null");
-			}
-		}
-
-		SelectTransformation<OUT> selectTransform = new SelectTransformation<OUT>(this.getTransformation(), Lists.newArrayList(outputNames));
-		return new DataStream<OUT>(this.getExecutionEnvironment(), selectTransform);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
new file mode 100644
index 0000000..11ee7f2
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
@@ -0,0 +1,62 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.transformations.SelectTransformation;
+import org.apache.flink.streaming.api.transformations.SplitTransformation;
+
+/**
+ * The SplitStream represents an operator that has been split using an
+ * {@link OutputSelector}. Named outputs can be selected using the
+ * {@link #select} function. To apply transformation on the whole output simply
+ * call the transformation on the SplitStream
+ *
+ * @param <OUT> The type of the elements in the Stream
+ */
+public class SplitStream<OUT> extends DataStream<OUT> {
+
+	protected SplitStream(DataStream<OUT> dataStream, OutputSelector<OUT> outputSelector) {
+		super(dataStream.getExecutionEnvironment(), new SplitTransformation<OUT>(dataStream.getTransformation(), outputSelector));
+	}
+
+	/**
+	 * Sets the output names for which the next operator will receive values.
+	 * 
+	 * @param outputNames
+	 *            The output names for which the operator will receive the
+	 *            input.
+	 * @return Returns the selected DataStream
+	 */
+	public DataStream<OUT> select(String... outputNames) {
+		return selectOutput(outputNames);
+	}
+
+	private DataStream<OUT> selectOutput(String[] outputNames) {
+		for (String outName : outputNames) {
+			if (outName == null) {
+				throw new RuntimeException("Selected names must not be null");
+			}
+		}
+
+		SelectTransformation<OUT> selectTransform = new SelectTransformation<OUT>(this.getTransformation(), Lists.newArrayList(outputNames));
+		return new DataStream<OUT>(this.getExecutionEnvironment(), selectTransform);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 55bf889..0b8482d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -40,7 +40,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.datastream.WindowedDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.WindowMapFunction;
@@ -457,7 +457,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 			}
 		};
 
-		SplitDataStream<Integer> split = unionFilter.split(outputSelector);
+		SplitStream<Integer> split = unionFilter.split(outputSelector);
 		split.select("dummy").addSink(new NoOpSink<Integer>());
 		List<OutputSelector<?>> outputSelectors = env.getStreamGraph().getStreamNode(unionFilter.getId()).getOutputSelectors();
 		assertEquals(1, outputSelectors.size());

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 7bdebf8..bd97e84 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.IterativeStream;
 import org.apache.flink.streaming.api.datastream.IterativeStream.ConnectedIterativeStreams;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
@@ -212,7 +212,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink<Integer>());
 		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
 
-		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
+		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
 				.map(NoOpIntMap).name("EvenOddSourceMap")
 				.split(new EvenOddOutputSelector());
 
@@ -295,7 +295,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 				.addSink(new ReceiveCheckNoOpSink<Integer>());
 		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
 
-		SplitDataStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
+		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
 				.map(NoOpIntMap)
 				.name("split")
 				.split(new EvenOddOutputSelector());

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
index 6401546..42febea 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
@@ -77,7 +77,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		DataStream<Tuple2<Integer, Integer>> sourceStream = env.addSource(new TupleSource(numElements, numKeys));
 
-		SplitDataStream<Tuple2<Integer, Integer>> splittedResult = sourceStream
+		SplitStream<Tuple2<Integer, Integer>> splittedResult = sourceStream
 			.keyBy(0)
 			.fold(0, new FoldFunction<Tuple2<Integer, Integer>, Integer>() {
 				@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
index b7df2ec..d2e24c9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
@@ -27,7 +27,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
@@ -102,7 +102,7 @@ public class DirectedOutputTest extends StreamingMultipleProgramsTestBase {
 		TestListResultSink<Long> evenAndOddSink = new TestListResultSink<Long>();
 		TestListResultSink<Long> allSink = new TestListResultSink<Long>();
 
-		SplitDataStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector());
+		SplitStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector());
 		source.select(EVEN).addSink(evenSink);
 		source.select(ODD, TEN).addSink(oddAndTenSink);
 		source.select(EVEN, ODD).addSink(evenAndOddSink);

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
index d35c9bd..5e46508 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.datastream.IterativeStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.WindowMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
@@ -138,7 +138,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 			}
 		}).iterate(5000);
 
-		SplitDataStream<Tuple2<Long, Tuple2<String, Long>>> step = it.map(new IncrementMap()).split(new
+		SplitStream<Tuple2<Long, Tuple2<String, Long>>> step = it.map(new IncrementMap()).split(new
 				MyOutputSelector());
 		it.closeWith(step.select("iterate"));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
index af19af7..2cf66b9 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.IterativeStream;
-import org.apache.flink.streaming.api.datastream.SplitDataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 
@@ -74,7 +74,7 @@ public class IterateExample {
 
 		// apply the step function to get the next Fibonacci number
 		// increment the counter and split the output with the output selector
-		SplitDataStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> step = it.map(new Step())
+		SplitStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> step = it.map(new Step())
 				.split(new MySelector());
 
 		// close the iteration by selecting the tuples that were directed to the

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 6ad7629..0cf1df8 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -729,15 +729,15 @@ class DataStream[T](javaStream: JavaStream[T]) {
    *
    * Operator used for directing tuples to specific named outputs using an
    * OutputSelector. Calling this method on an operator creates a new
-   * SplitDataStream.
+   * [[SplitStream]].
    */
-  def split(selector: OutputSelector[T]): SplitDataStream[T] = javaStream.split(selector)
+  def split(selector: OutputSelector[T]): SplitStream[T] = javaStream.split(selector)
 
   /**
-   * Creates a new SplitDataStream that contains only the elements satisfying the
+   * Creates a new [[SplitStream]] that contains only the elements satisfying the
    *  given output selector predicate.
    */
-  def split(fun: T => TraversableOnce[String]): SplitDataStream[T] = {
+  def split(fun: T => TraversableOnce[String]): SplitStream[T] = {
     if (fun == null) {
       throw new NullPointerException("OutputSelector must not be null.")
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala
deleted file mode 100644
index 105d2c1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitDataStream.scala
+++ /dev/null
@@ -1,37 +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.scala
-
-import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaStream }
-
-/**
- * The SplitDataStream represents an operator that has been split using an
- * {@link OutputSelector}. Named outputs can be selected using the
- * {@link #select} function. To apply a transformation on the whole output simply call
- * the appropriate method on this stream.
- *
- */
-class SplitDataStream[T](javaStream: SplitJavaStream[T]) extends DataStream[T](javaStream){
-
-  /**
-   *  Sets the output names for which the next operator will receive values.
-   */
-  def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*)
-  
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
new file mode 100644
index 0000000..deea6f0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.scala
+
+import org.apache.flink.streaming.api.datastream.{ SplitStream => SplitJavaStream }
+
+/**
+ * The SplitStream represents an operator that has been split using an
+ * {@link OutputSelector}. Named outputs can be selected using the
+ * {@link #select} function. To apply a transformation on the whole output simply call
+ * the appropriate method on this stream.
+ *
+ */
+class SplitStream[T](javaStream: SplitJavaStream[T]) extends DataStream[T](javaStream){
+
+  /**
+   *  Sets the output names for which the next operator will receive values.
+   */
+  def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*)
+  
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
index 625678a..d65ea41 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
@@ -25,7 +25,7 @@ import org.apache.flink.api.scala.{createTuple2TypeInformation => apiTupleCreato
 import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
 import org.apache.flink.streaming.api.datastream.{ WindowedDataStream => JavaWStream }
-import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaStream }
+import org.apache.flink.streaming.api.datastream.{ SplitStream => SplitJavaStream }
 import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => ConnectedJavaStreams }
 import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream }
 import language.implicitConversions
@@ -44,8 +44,8 @@ package object scala {
   implicit def javaToScalaWindowedStream[R](javaWStream: JavaWStream[R]): WindowedDataStream[R] =
     new WindowedDataStream[R](javaWStream)
 
-  implicit def javaToScalaSplitStream[R](javaStream: SplitJavaStream[R]): SplitDataStream[R] =
-    new SplitDataStream[R](javaStream)
+  implicit def javaToScalaSplitStream[R](javaStream: SplitJavaStream[R]): SplitStream[R] =
+    new SplitStream[R](javaStream)
 
   implicit def javaToScalaConnectedStream[IN1, IN2](javaStream: ConnectedJavaStreams[IN1, IN2]):
   ConnectedStreams[IN1, IN2] = new ConnectedStreams[IN1, IN2](javaStream)

http://git-wip-us.apache.org/repos/asf/flink/blob/9513f0e3/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index 6ecdb85..d1fd233 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -118,9 +118,9 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       classOf[ConnectedStreams[_,_]])
 
     checkMethods(
-      "SplitDataStream", "SplitDataStream",
-      classOf[org.apache.flink.streaming.api.datastream.SplitDataStream[_]],
-      classOf[SplitDataStream[_]])
+      "SplitStream", "SplitStream",
+      classOf[org.apache.flink.streaming.api.datastream.SplitStream[_]],
+      classOf[SplitStream[_]])
 
     checkMethods(
       "WindowedStream", "WindowedStream",


[09/13] flink git commit: [FLINK-2550] Remove groupBy and GroupedDataStream

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
index abc1a18..6c439b9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
@@ -194,7 +194,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		DataStream<OuterPojo> sourceStream22 = env.addSource(new PojoSource());
 
 		sourceStream21
-				.groupBy(2, 2)
+				.keyBy(2, 2)
 				.window(Time.of(10, new MyTimestamp(), 0))
 				.every(Time.of(4, new MyTimestamp(), 0))
 				.maxBy(3)
@@ -260,7 +260,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 				return new Tuple2<Long, Integer>(value, 1);
 			}
 		})
-				.groupBy(0)
+				.keyBy(0)
 				.window(Count.of(10000)).sum(1).flatten()
 				.filter(new FilterFunction<Tuple2<Long, Integer>>() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
index 512a0df..19a61ba 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
@@ -179,7 +179,7 @@ public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
 			public void flatMap(Integer value, Collector<String> out) throws Exception {
 				out.collect("x " + value);
 			}
-		}).groupBy(new KeySelector<String, Integer>() {
+		}).keyBy(new KeySelector<String, Integer>() {
 
 			private static final long serialVersionUID = 1L;
 
@@ -197,7 +197,7 @@ public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
 			public Long map(Integer value) throws Exception {
 				return Long.valueOf(value + 1);
 			}
-		}).groupBy(new KeySelector<Long, Long>() {
+		}).keyBy(new KeySelector<Long, Long>() {
 
 			private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
index b762d65..2c06c00 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
@@ -69,7 +69,7 @@ public class ParallelMergeITCase extends StreamingProgramTestBase {
 		DataStream<Tuple2<String, Integer>> counts =
 				text.flatMap(new Tokenizer())
 						.window(Time.of(1000, TimeUnit.MILLISECONDS))
-						.groupBy(0)
+						.keyBy(0)
 						.sum(1)
 						.flatten();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
index db09373..4611966 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
@@ -107,15 +107,15 @@ public class WindowingITCase extends StreamingMultipleProgramsTestBase {
 		source.window(Time.of(3, ts, 1)).every(Time.of(2, ts, 1)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink1());
 
-		source.window(Time.of(4, ts, 1)).groupBy(new ModKey(2))
+		source.window(Time.of(4, ts, 1)).keyBy(new ModKey(2))
 				.mapWindow(new IdentityWindowMap())
 				.flatten()
 				.addSink(new TestSink2()).name("TESTSIUNK2");
 
-		source.groupBy(key).window(Time.of(4, ts, 1)).sum(0).getDiscretizedStream()
+		source.keyBy(key).window(Time.of(4, ts, 1)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink4());
 
-		source.groupBy(new ModKey(3)).window(Count.of(2)).groupBy(new ModKey(2))
+		source.keyBy(new ModKey(3)).window(Count.of(2)).keyBy(new ModKey(2))
 				.mapWindow(new IdentityWindowMap())
 				.flatten()
 				.addSink(new TestSink5());
@@ -123,14 +123,14 @@ public class WindowingITCase extends StreamingMultipleProgramsTestBase {
 		source.window(Time.of(2, ts)).every(Time.of(3, ts)).min(0).getDiscretizedStream()
 				.addSink(new TestSink3());
 
-		source.groupBy(key).window(Time.of(4, ts, 1)).max(0).getDiscretizedStream()
+		source.keyBy(key).window(Time.of(4, ts, 1)).max(0).getDiscretizedStream()
 				.addSink(new TestSink6());
 
 		source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap())
 				.flatten()
 				.addSink(new TestSink7());
 
-		source.window(Time.of(5, ts, 1)).every(Time.of(4, ts, 1)).groupBy(new ModKey(2)).sum(0)
+		source.window(Time.of(5, ts, 1)).every(Time.of(4, ts, 1)).keyBy(new ModKey(2)).sum(0)
 				.getDiscretizedStream()
 				.addSink(new TestSink8());
 
@@ -152,7 +152,7 @@ public class WindowingITCase extends StreamingMultipleProgramsTestBase {
 
 		source.every(Count.of(4)).sum(0).getDiscretizedStream().addSink(new TestSink11());
 
-		source.window(FullStream.window()).every(Count.of(4)).groupBy(key).sum(0)
+		source.window(FullStream.window()).every(Count.of(4)).keyBy(key).sum(0)
 				.getDiscretizedStream()
 				.addSink(new TestSink12());
 
@@ -197,7 +197,7 @@ public class WindowingITCase extends StreamingMultipleProgramsTestBase {
 
 		source2.window(Time.of(2, ts, 1)).sum(0).getDiscretizedStream().addSink(new TestSink9());
 
-		source3.window(Time.of(5, ts, 1)).groupBy(new ModKey(2)).sum(0).getDiscretizedStream()
+		source3.window(Time.of(5, ts, 1)).keyBy(new ModKey(2)).sum(0).getDiscretizedStream()
 				.addSink(new TestSink10());
 
 		source

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

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
index bf96cc1..49876ec 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
@@ -38,7 +38,7 @@ public class SocketOutputFormatITCase extends SocketOutputTestBase {
 
 		DataStream<String> counts =
 				text.flatMap(new CsvOutputFormatITCase.Tokenizer())
-						.groupBy(0).sum(1).map(new MapFunction<Tuple2<String, Integer>, String>() {
+						.keyBy(0).sum(1).map(new MapFunction<Tuple2<String, Integer>, String>() {
 					@Override
 					public String map(Tuple2<String, Integer> value) throws Exception {
 						return value.toString() + "\n";

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

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
index 1473097..17add2c 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
@@ -66,7 +66,7 @@ public class SocketTextStreamWordCount {
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				text.flatMap(new Tokenizer())
 						// group by the tuple field "0" and sum up tuple field "1"
-						.groupBy(0)
+						.keyBy(0)
 						.sum(1);
 
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
index 32a2dfe..c2477b5 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
@@ -69,7 +69,7 @@ public class TwitterStream {
 				// selecting English tweets and splitting to (word, 1)
 				.flatMap(new SelectEnglishAndTokenizeFlatMap())
 				// group by words and sum their occurrences
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
index f8d8652..4730cc1 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
@@ -76,7 +76,7 @@ public class SessionWindowing {
 				});
 
 		// We create sessions for each id with max timeout of 3 time units
-		DataStream<Tuple3<String, Long, Integer>> aggregated = source.groupBy(0)
+		DataStream<Tuple3<String, Long, Integer>> aggregated = source.keyBy(0)
 				.window(new SessionTriggerPolicy(3L),
 						new TumblingEvictionPolicy<Tuple3<String, Long, Integer>>()).sum(2)
 				.flatten();

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
index 1b48387..55d48dd 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
@@ -60,7 +60,7 @@ public class TopSpeedWindowing {
 		} else {
 			carData = env.addSource(CarSource.create(numOfCars));
 		}
-		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData.groupBy(0)
+		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData.keyBy(0)
 				.window(Time.of(evictionSec * 1000, new CarTimestamp()))
 				.every(Delta.of(triggerMeters,
 						new DeltaFunction<Tuple4<Integer, Integer, Double, Long>>() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
index bd3acc6..023a36a 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
@@ -72,7 +72,7 @@ public class WindowWordCount {
 				// create windows of windowSize records slided every slideSize records
 				.window(Count.of(windowSize)).every(Count.of(slideSize))
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1)
+				.keyBy(0).sum(1)
 				// flatten the windows to a single stream
 				.flatten();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
index 5ff3fc1..591ef51 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
@@ -63,7 +63,7 @@ public class PojoExample {
 		// split up the lines into Word objects
 		text.flatMap(new Tokenizer())
 		// group by the field word and sum up the frequency
-				.groupBy("word").sum("frequency");
+				.keyBy("word").sum("frequency");
 
 		if (fileOutput) {
 			counts.writeAsText(outputPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
index c207d60..a594c94 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
@@ -67,7 +67,7 @@ public class WordCount {
 		// split up the lines in pairs (2-tuples) containing: (word,1)
 		text.flatMap(new Tokenizer())
 		// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
index 4dbc3fb..9ec17d4 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
@@ -57,7 +57,7 @@ object SocketTextStreamWordCount {
     val text = env.socketTextStream(hostName, port)
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
-      .groupBy(0)
+      .keyBy(0)
       .sum(1)
 
     if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
index 9603f71..8e3c7d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
@@ -56,7 +56,7 @@ object TopSpeedWindowing {
 
     val cars = setCarsInput(env)
 
-    val topSeed = cars.groupBy("carId")
+    val topSeed = cars.keyBy("carId")
       .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
       .every(Delta.of[CarEvent](triggerMeters,
           (oldSp,newSp) => newSp.distance-oldSp.distance, CarEvent(0,0,0,0)))

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
index 41c1a7a..4727cc5 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
@@ -169,8 +169,8 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
    * second input stream.
    * @return @return The transformed { @link ConnectedStreams}
    */
-  def groupBy(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
-    javaStream.groupBy(keyPosition1, keyPosition2)
+  def keyBy(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
+    javaStream.keyBy(keyPosition1, keyPosition2)
   }
 
   /**
@@ -185,9 +185,9 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
    * The fields used to group the second input stream.
    * @return @return The transformed { @link ConnectedStreams}
    */
-  def groupBy(keyPositions1: Array[Int], keyPositions2: Array[Int]): 
+  def keyBy(keyPositions1: Array[Int], keyPositions2: Array[Int]):
   ConnectedStreams[IN1, IN2] = {
-    javaStream.groupBy(keyPositions1, keyPositions2)
+    javaStream.keyBy(keyPositions1, keyPositions2)
   }
 
   /**
@@ -203,8 +203,8 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
    * The grouping expression for the second input
    * @return The grouped { @link ConnectedStreams}
    */
-  def groupBy(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
-    javaStream.groupBy(field1, field2)
+  def keyBy(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
+    javaStream.keyBy(field1, field2)
   }
 
   /**
@@ -221,9 +221,9 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
    * The grouping expressions for the second input
    * @return The grouped { @link ConnectedStreams}
    */
-  def groupBy(fields1: Array[String], fields2: Array[String]): 
+  def keyBy(fields1: Array[String], fields2: Array[String]):
   ConnectedStreams[IN1, IN2] = {
-    javaStream.groupBy(fields1, fields2)
+    javaStream.keyBy(fields1, fields2)
   }
 
   /**
@@ -238,7 +238,7 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
    * The function used for grouping the second input
    * @return The grouped { @link ConnectedStreams}
    */
-  def groupBy[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
+  def keyBy[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
   ConnectedStreams[IN1, IN2] = {
 
     val cleanFun1 = clean(fun1)
@@ -250,7 +250,7 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
       def getKey(in: IN2) = cleanFun2(in)
     }
 
-    javaStream.groupBy(keyExtractor1, keyExtractor2)
+    javaStream.keyBy(keyExtractor1, keyExtractor2)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index c9aee61..07828db 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -209,58 +209,31 @@ class DataStream[T](javaStream: JavaStream[T]) {
    */
   def connect[T2](dataStream: DataStream[T2]): ConnectedStreams[T, T2] =
     javaStream.connect(dataStream.getJavaStream)
-
-
-
-  /**
-   * Partitions the operator states of the DataStream by the given key positions 
-   * (for tuple/array types).
-   */
-  def keyBy(fields: Int*): DataStream[T] = javaStream.keyBy(fields: _*)
-
-  /**
-   *
-   * Partitions the operator states of the DataStream by the given field expressions.
-   */
-  def keyBy(firstField: String, otherFields: String*): DataStream[T] =
-    javaStream.keyBy(firstField +: otherFields.toArray: _*)
-
-
-  /**
-   * Partitions the operator states of the DataStream by the given K key. 
-   */
-  def keyBy[K: TypeInformation](fun: T => K): DataStream[T] = {
-    val cleanFun = clean(fun)
-    val keyExtractor = new KeySelector[T, K] {
-      def getKey(in: T) = cleanFun(in)
-    }
-    javaStream.keyBy(keyExtractor)
-  }
   
   /**
    * Groups the elements of a DataStream by the given key positions (for tuple/array types) to
    * be used with grouped operators like grouped reduce or grouped aggregations.
    */
-  def groupBy(fields: Int*): GroupedDataStream[T, JavaTuple] = javaStream.groupBy(fields: _*)
+  def keyBy(fields: Int*): KeyedStream[T, JavaTuple] = javaStream.keyBy(fields: _*)
 
   /**
    * Groups the elements of a DataStream by the given field expressions to
    * be used with grouped operators like grouped reduce or grouped aggregations.
    */
-  def groupBy(firstField: String, otherFields: String*): GroupedDataStream[T, JavaTuple] =
-   javaStream.groupBy(firstField +: otherFields.toArray: _*)   
+  def keyBy(firstField: String, otherFields: String*): KeyedStream[T, JavaTuple] =
+   javaStream.keyBy(firstField +: otherFields.toArray: _*)   
   
   /**
    * Groups the elements of a DataStream by the given K key to
    * be used with grouped operators like grouped reduce or grouped aggregations.
    */
-  def groupBy[K: TypeInformation](fun: T => K): GroupedDataStream[T, K] = {
+  def keyBy[K: TypeInformation](fun: T => K): KeyedStream[T, K] = {
 
     val cleanFun = clean(fun)
     val keyExtractor = new KeySelector[T, K] {
       def getKey(in: T) = cleanFun(in)
     }
-    javaStream.groupBy(keyExtractor)
+    javaStream.keyBy(keyExtractor)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/GroupedDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/GroupedDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/GroupedDataStream.scala
deleted file mode 100644
index e1a963d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/GroupedDataStream.scala
+++ /dev/null
@@ -1,196 +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.scala
-
-import org.apache.flink.streaming.api.datastream.{ GroupedDataStream => GroupedJavaStream, DataStream => JavaStream }
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
-import org.apache.flink.streaming.api.operators.StreamGroupedReduce
-import scala.reflect.ClassTag
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.functions.FoldFunction
-import org.apache.flink.api.common.functions.ReduceFunction
-
-
-class GroupedDataStream[T, K](javaStream: GroupedJavaStream[T, K]) 
-  extends DataStream[T](javaStream) {
- 
-  /**
-   * Creates a new [[DataStream]] by reducing the elements of this DataStream
-   * using an associative reduce function. An independent aggregate is kept per key.
-   */
-  def reduce(reducer: ReduceFunction[T]): DataStream[T] = {
-    if (reducer == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
- 
-    javaStream.reduce(reducer)
-  }
-
-  /**
-   * Creates a new [[DataStream]] by reducing the elements of this DataStream
-   * using an associative reduce function. An independent aggregate is kept per key.
-   */
-  def reduce(fun: (T, T) => T): DataStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
-    }
-    reduce(reducer)
-  }
-
-  /**
-   * Creates a new [[DataStream]] by folding the elements of this DataStream
-   * using an associative fold function and an initial value. An independent 
-   * aggregate is kept per key.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, folder: FoldFunction[T,R]): 
-  DataStream[R] = {
-    if (folder == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
-    
-    javaStream.fold(initialValue, folder).
-      returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Creates a new [[DataStream]] by folding the elements of this DataStream
-   * using an associative fold function and an initial value. An independent 
-   * aggregate is kept per key.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, fun: (R,T) => R): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val folder = new FoldFunction[T,R] {
-      def fold(acc: R, v: T) = {
-        cleanFun(acc, v)
-      }
-    }
-    fold(initialValue, folder)
-  }
-  
-  /**
-   * Applies an aggregation that that gives the current maximum of the data stream at
-   * the given position by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
-  
-  /**
-   * Applies an aggregation that that gives the current maximum of the data stream at
-   * the given field by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
-  
-  /**
-   * Applies an aggregation that that gives the current minimum of the data stream at
-   * the given position by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
-  
-  /**
-   * Applies an aggregation that that gives the current minimum of the data stream at
-   * the given field by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
-
-  /**
-   * Applies an aggregation that sums the data stream at the given position by the given 
-   * key. An independent aggregate is kept per key.
-   *
-   */
-  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
-  
-  /**
-   * Applies an aggregation that sums the data stream at the given field by the given 
-   * key. An independent aggregate is kept per key.
-   *
-   */
-  def sum(field: String): DataStream[T] =  aggregate(AggregationType.SUM, field)
-
-  /**
-   * Applies an aggregation that that gives the current minimum element of the data stream by
-   * the given position by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the minimal value.
-   *
-   */
-  def minBy(position: Int): DataStream[T] = aggregate(AggregationType
-    .MINBY, position)
-    
-   /**
-   * Applies an aggregation that that gives the current minimum element of the data stream by
-   * the given field by the given key. An independent aggregate is kept per key.
-   * When equality, the first element is returned with the minimal value.
-   *
-   */
-  def minBy(field: String): DataStream[T] = aggregate(AggregationType
-    .MINBY, field )
-
-   /**
-   * Applies an aggregation that that gives the current maximum element of the data stream by
-   * the given position by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the maximal value.
-   *
-   */
-  def maxBy(position: Int): DataStream[T] =
-    aggregate(AggregationType.MAXBY, position)
-    
-   /**
-   * Applies an aggregation that that gives the current maximum element of the data stream by
-   * the given field by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the maximal value.
-   *
-   */
-  def maxBy(field: String): DataStream[T] =
-    aggregate(AggregationType.MAXBY, field)
-    
-  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
-    val position = fieldNames2Indices(javaStream.getType(), Array(field))(0)
-    aggregate(aggregationType, position)
-  }
-
-  private def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
-
-    val reducer = aggregationType match {
-      case AggregationType.SUM =>
-        new SumAggregator(position, javaStream.getType, javaStream.getExecutionConfig)
-      case _ =>
-        new ComparableAggregator(position, javaStream.getType, aggregationType, true,
-          javaStream.getExecutionConfig)
-    }
-
-    val invokable =  new StreamGroupedReduce[T](reducer,javaStream.getKeySelector())
-     
-    new DataStream[T](javaStream.transform("aggregation", javaStream.getType(),invokable))
-      .asInstanceOf[DataStream[T]]
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
new file mode 100644
index 0000000..25244cd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -0,0 +1,195 @@
+/*
+ * 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.scala
+
+import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream, DataStream => JavaStream }
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
+import org.apache.flink.streaming.api.operators.StreamGroupedReduce
+import scala.reflect.ClassTag
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.functions.FoldFunction
+import org.apache.flink.api.common.functions.ReduceFunction
+
+
+class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T](javaStream) {
+ 
+  /**
+   * Creates a new [[DataStream]] by reducing the elements of this DataStream
+   * using an associative reduce function. An independent aggregate is kept per key.
+   */
+  def reduce(reducer: ReduceFunction[T]): DataStream[T] = {
+    if (reducer == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+ 
+    javaStream.reduce(reducer)
+  }
+
+  /**
+   * Creates a new [[DataStream]] by reducing the elements of this DataStream
+   * using an associative reduce function. An independent aggregate is kept per key.
+   */
+  def reduce(fun: (T, T) => T): DataStream[T] = {
+    if (fun == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+    val cleanFun = clean(fun)
+    val reducer = new ReduceFunction[T] {
+      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
+    }
+    reduce(reducer)
+  }
+
+  /**
+   * Creates a new [[DataStream]] by folding the elements of this DataStream
+   * using an associative fold function and an initial value. An independent 
+   * aggregate is kept per key.
+   */
+  def fold[R: TypeInformation: ClassTag](initialValue: R, folder: FoldFunction[T,R]): 
+  DataStream[R] = {
+    if (folder == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+    
+    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
+    
+    javaStream.fold(initialValue, folder).
+      returns(outType).asInstanceOf[JavaStream[R]]
+  }
+
+  /**
+   * Creates a new [[DataStream]] by folding the elements of this DataStream
+   * using an associative fold function and an initial value. An independent 
+   * aggregate is kept per key.
+   */
+  def fold[R: TypeInformation: ClassTag](initialValue: R, fun: (R,T) => R): DataStream[R] = {
+    if (fun == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+    val cleanFun = clean(fun)
+    val folder = new FoldFunction[T,R] {
+      def fold(acc: R, v: T) = {
+        cleanFun(acc, v)
+      }
+    }
+    fold(initialValue, folder)
+  }
+  
+  /**
+   * Applies an aggregation that that gives the current maximum of the data stream at
+   * the given position by the given key. An independent aggregate is kept per key.
+   *
+   */
+  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
+  
+  /**
+   * Applies an aggregation that that gives the current maximum of the data stream at
+   * the given field by the given key. An independent aggregate is kept per key.
+   *
+   */
+  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
+  
+  /**
+   * Applies an aggregation that that gives the current minimum of the data stream at
+   * the given position by the given key. An independent aggregate is kept per key.
+   *
+   */
+  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
+  
+  /**
+   * Applies an aggregation that that gives the current minimum of the data stream at
+   * the given field by the given key. An independent aggregate is kept per key.
+   *
+   */
+  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
+
+  /**
+   * Applies an aggregation that sums the data stream at the given position by the given 
+   * key. An independent aggregate is kept per key.
+   *
+   */
+  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
+  
+  /**
+   * Applies an aggregation that sums the data stream at the given field by the given 
+   * key. An independent aggregate is kept per key.
+   *
+   */
+  def sum(field: String): DataStream[T] =  aggregate(AggregationType.SUM, field)
+
+  /**
+   * Applies an aggregation that that gives the current minimum element of the data stream by
+   * the given position by the given key. An independent aggregate is kept per key. 
+   * When equality, the first element is returned with the minimal value.
+   *
+   */
+  def minBy(position: Int): DataStream[T] = aggregate(AggregationType
+    .MINBY, position)
+    
+   /**
+   * Applies an aggregation that that gives the current minimum element of the data stream by
+   * the given field by the given key. An independent aggregate is kept per key.
+   * When equality, the first element is returned with the minimal value.
+   *
+   */
+  def minBy(field: String): DataStream[T] = aggregate(AggregationType
+    .MINBY, field )
+
+   /**
+   * Applies an aggregation that that gives the current maximum element of the data stream by
+   * the given position by the given key. An independent aggregate is kept per key. 
+   * When equality, the first element is returned with the maximal value.
+   *
+   */
+  def maxBy(position: Int): DataStream[T] =
+    aggregate(AggregationType.MAXBY, position)
+    
+   /**
+   * Applies an aggregation that that gives the current maximum element of the data stream by
+   * the given field by the given key. An independent aggregate is kept per key. 
+   * When equality, the first element is returned with the maximal value.
+   *
+   */
+  def maxBy(field: String): DataStream[T] =
+    aggregate(AggregationType.MAXBY, field)
+    
+  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
+    val position = fieldNames2Indices(javaStream.getType(), Array(field))(0)
+    aggregate(aggregationType, position)
+  }
+
+  private def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
+
+    val reducer = aggregationType match {
+      case AggregationType.SUM =>
+        new SumAggregator(position, javaStream.getType, javaStream.getExecutionConfig)
+      case _ =>
+        new ComparableAggregator(position, javaStream.getType, aggregationType, true,
+          javaStream.getExecutionConfig)
+    }
+
+    val invokable =  new StreamGroupedReduce[T](reducer,javaStream.getKeySelector())
+     
+    new DataStream[T](javaStream.transform("aggregation", javaStream.getType(),invokable))
+      .asInstanceOf[DataStream[T]]
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
index f584767..e0bbaf8 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
@@ -151,7 +151,7 @@ object StreamJoinOperator {
     private def createJoinOperator(): JavaStream[(I1, I2)] = {
 
 //      val returnType = createTuple2TypeInformation[I1, I2](op.input1.getType, op.input2.getType)
-//      op.input1.groupBy(keys1).connect(op.input2.groupBy(keys2))
+//      op.input1.keyBy(keys1).connect(op.input2.keyBy(keys2))
 //        .addGeneralWindowCombine(getJoinWindowFunction(this, (_, _)),
 //          returnType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)
       null
@@ -172,7 +172,7 @@ object StreamJoinOperator {
 
       val cleanFun = clean(getJoinWindowFunction(jp, fun))
 
-//      op.input1.groupBy(jp.keys1).connect(op.input2.groupBy(jp.keys2))
+//      op.input1.keyBy(jp.keys1).connect(op.input2.keyBy(jp.keys2))
 //        .addGeneralWindowCombine[R](
 //          cleanFun,
 //          implicitly[TypeInformation[R]],

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
index f4d2154..8ef94f0 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
@@ -80,7 +80,7 @@ class WindowedDataStream[T](javaStream: JavaWStream[T]) {
    * DataStream.window(...) operator on an already grouped data stream.
    *
    */
-  def groupBy(fields: Int*): WindowedDataStream[T] = javaStream.groupBy(fields: _*)
+  def keyBy(fields: Int*): WindowedDataStream[T] = javaStream.keyBy(fields: _*)
 
   /**
    * Groups the elements of the WindowedDataStream using the given
@@ -91,8 +91,8 @@ class WindowedDataStream[T](javaStream: JavaWStream[T]) {
    * DataStream.window(...) operator on an already grouped data stream.
    *
    */
-  def groupBy(firstField: String, otherFields: String*): WindowedDataStream[T] =
-   javaStream.groupBy(firstField +: otherFields.toArray: _*)   
+  def keyBy(firstField: String, otherFields: String*): WindowedDataStream[T] =
+   javaStream.keyBy(firstField +: otherFields.toArray: _*)
     
   /**
    * Groups the elements of the WindowedDataStream using the given
@@ -103,13 +103,13 @@ class WindowedDataStream[T](javaStream: JavaWStream[T]) {
    * DataStream.window(...) operator on an already grouped data stream.
    *
    */
-  def groupBy[K: TypeInformation](fun: T => K): WindowedDataStream[T] = {
+  def keyBy[K: TypeInformation](fun: T => K): WindowedDataStream[T] = {
 
     val cleanFun = clean(fun)
     val keyExtractor = new KeySelector[T, K] {
       def getKey(in: T) = cleanFun(in)
     }
-    javaStream.groupBy(keyExtractor)
+    javaStream.keyBy(keyExtractor)
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
index b8a3b94..625678a 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
@@ -26,8 +26,8 @@ import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
 import org.apache.flink.streaming.api.datastream.{ WindowedDataStream => JavaWStream }
 import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaStream }
-import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => JavaConStream }
-import org.apache.flink.streaming.api.datastream.{ GroupedDataStream => GroupedJavaStream }
+import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => ConnectedJavaStreams }
+import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream }
 import language.implicitConversions
 
 package object scala {
@@ -38,8 +38,8 @@ package object scala {
   implicit def javaToScalaStream[R](javaStream: JavaStream[R]): DataStream[R] =
     new DataStream[R](javaStream)
     
-  implicit def javaToScalaGroupedStream[R, K](javaStream: GroupedJavaStream[R, K]): 
-  GroupedDataStream[R, K] = new GroupedDataStream[R, K](javaStream)    
+  implicit def javaToScalaGroupedStream[R, K](javaStream: KeyedJavaStream[R, K]):
+  KeyedStream[R, K] = new KeyedStream[R, K](javaStream)
 
   implicit def javaToScalaWindowedStream[R](javaWStream: JavaWStream[R]): WindowedDataStream[R] =
     new WindowedDataStream[R](javaWStream)
@@ -47,7 +47,7 @@ package object scala {
   implicit def javaToScalaSplitStream[R](javaStream: SplitJavaStream[R]): SplitDataStream[R] =
     new SplitDataStream[R](javaStream)
 
-  implicit def javaToScalaConnectedStream[IN1, IN2](javaStream: JavaConStream[IN1, IN2]):
+  implicit def javaToScalaConnectedStream[IN1, IN2](javaStream: ConnectedJavaStreams[IN1, IN2]):
   ConnectedStreams[IN1, IN2] = new ConnectedStreams[IN1, IN2](javaStream)
 
   implicit def seqToFlinkSource[T: ClassTag: TypeInformation](scalaSeq: Seq[T]) : DataStream[T] =

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index 606aac5..5a5a8c9 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -50,7 +50,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert("testMap" == dataStream1.getName)
 
     val dataStream2 = env.generateSequence(0, 0).name("testSource2")
-      .groupBy(x=>x)
+      .keyBy(x=>x)
       .reduce((x, y) => 0)
       .name("testReduce")
     assert("testReduce" == dataStream2.getName)
@@ -83,7 +83,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
   }
 
   /**
-   * Tests that {@link DataStream#groupBy} and {@link DataStream#partitionBy(KeySelector)} result in
+   * Tests that {@link DataStream#keyBy} and {@link DataStream#partitionBy(KeySelector)} result in
    * different and correct topologies. Does the some for the {@link ConnectedStreams}.
    */
   @Test
@@ -95,10 +95,10 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
     val connected = src1.connect(src2)
 
-    val group1 = src1.groupBy(0)
-    val group2 = src1.groupBy(1, 0)
-    val group3 = src1.groupBy("_1")
-    val group4 = src1.groupBy(x => x._1)
+    val group1 = src1.keyBy(0)
+    val group2 = src1.keyBy(1, 0)
+    val group3 = src1.keyBy("_1")
+    val group4 = src1.keyBy(x => x._1)
 
     val gid1 = createDownStreamId(group1)
     val gid2 = createDownStreamId(group2)
@@ -145,20 +145,20 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid3)))
 
     //Testing ConnectedStreams grouping
-    val connectedGroup1: ConnectedStreams[_, _] = connected.groupBy(0, 0)
+    val connectedGroup1: ConnectedStreams[_, _] = connected.keyBy(0, 0)
     val downStreamId1: Integer = createDownStreamId(connectedGroup1)
 
-    val connectedGroup2: ConnectedStreams[_, _] = connected.groupBy(Array[Int](0), Array[Int](0))
+    val connectedGroup2: ConnectedStreams[_, _] = connected.keyBy(Array[Int](0), Array[Int](0))
     val downStreamId2: Integer = createDownStreamId(connectedGroup2)
 
-    val connectedGroup3: ConnectedStreams[_, _] = connected.groupBy("_1", "_1")
+    val connectedGroup3: ConnectedStreams[_, _] = connected.keyBy("_1", "_1")
     val downStreamId3: Integer = createDownStreamId(connectedGroup3)
 
     val connectedGroup4: ConnectedStreams[_, _] =
-      connected.groupBy(Array[String]("_1"), Array[String]("_1"))
+      connected.keyBy(Array[String]("_1"), Array[String]("_1"))
     val downStreamId4: Integer = createDownStreamId(connectedGroup4)
 
-    val connectedGroup5: ConnectedStreams[_, _] = connected.groupBy(x => x._1, x => x._1)
+    val connectedGroup5: ConnectedStreams[_, _] = connected.keyBy(x => x._1, x => x._1)
     val downStreamId5: Integer = createDownStreamId(connectedGroup5)
 
     assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId1)))
@@ -413,10 +413,10 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     val foldFunction = new FoldFunction[Int, String] {
       override def fold(accumulator: String, value: Int): String = ""
     }
-    val fold = map.groupBy(x=>x).fold("", foldFunction)
+    val fold = map.keyBy(x=>x).fold("", foldFunction)
     assert(foldFunction == getFunctionForDataStream(fold))
     assert(
-      getFunctionForDataStream(map.groupBy(x=>x)
+      getFunctionForDataStream(map.keyBy(x=>x)
         .fold("", (x: String, y: Int) => ""))
         .isInstanceOf[FoldFunction[_, _]])
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
index 0d573a9..e09f164 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
@@ -36,7 +36,7 @@ object OutputFormatTestPrograms {
     val text = env.fromElements(input)
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
-      .groupBy(0)
+      .keyBy(0)
       .sum(1)
 
     counts.writeAsText(outputPath)
@@ -51,7 +51,7 @@ object OutputFormatTestPrograms {
     val text = env.fromElements(input)
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
-      .groupBy(0)
+      .keyBy(0)
       .sum(1)
 
     counts.writeAsCsv(outputPath)
@@ -66,7 +66,7 @@ object OutputFormatTestPrograms {
     val text = env.fromElements(input)
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
-      .groupBy(0)
+      .keyBy(0)
       .sum(1)
       .map(tuple => tuple.toString() + "\n")
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
index d5e2b7b..2131026 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
@@ -79,7 +79,7 @@ class StreamingOperatorsITCase extends ScalaStreamingMultipleProgramsTestBase {
     })
 
     val splittedResult = sourceStream
-      .groupBy(0)
+      .keyBy(0)
       .fold(0, new FoldFunction[(Int, Int), Int] {
         override def fold(accumulator: Int, value: (Int, Int)): Int = {
           accumulator + value._2

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
index 706b8dd..e5a1c23 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
@@ -88,7 +88,7 @@ public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 				.map(new StatefulCounterFunction())
 
 				// -------------- fourth vertex - reducer (failing) and the sink ----------------
-				.groupBy("prefix")
+				.keyBy("prefix")
 				.reduce(new OnceFailingReducer(NUM_STRINGS))
 				.addSink(new SinkFunction<PrefixCount>() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
index aa3e9e4..108e1e6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
@@ -59,7 +59,7 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
 		DataStream<Integer> stream2 = env.addSource(new IntGeneratingSourceFunction(NUM_STRINGS / 2));
 
 		stream1.union(stream2)
-				.groupBy(new IdentityKeySelector<Integer>())
+				.keyBy(new IdentityKeySelector<Integer>())
 				.map(new OnceFailingPartitionedSum(NUM_STRINGS))
 				.keyBy(0)
 				.addSink(new CounterSink());

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
index de8ee9d..270cfaa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
@@ -89,7 +89,7 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
 				.startNewChain()
 
 				// -------------- fourth vertex - reducer and the sink ----------------
-				.groupBy(0)
+				.keyBy(0)
 				.reduce(new OnceFailingReducer(NUM_LONGS))
 				.addSink(new SinkFunction<Tuple1<Long>>() {
 					@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
index 882634b..0804d53 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
@@ -70,7 +70,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 				.map(new StatefulCounterFunction())
 
 						// -------------- third vertex - counter and the sink ----------------
-				.groupBy("prefix")
+				.keyBy("prefix")
 				.map(new OnceFailingPrefixCounter(NUM_STRINGS))
 				.addSink(new SinkFunction<PrefixCount>() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
index 76c8e54..a19d8f2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
@@ -42,7 +42,7 @@ public class StreamingProgram {
 		DataStream<String> text = env.fromElements(WordCountData.TEXT).rebalance();
 
 		DataStream<Word> counts =
-				text.flatMap(new Tokenizer()).groupBy("word").sum("frequency");
+				text.flatMap(new Tokenizer()).keyBy("word").sum("frequency");
 
 		counts.addSink(new NoOpSink());
 


[08/13] flink git commit: Disable tests because operators are not implemented yet

Posted by al...@apache.org.
Disable tests because operators are not implemented yet


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

Branch: refs/heads/master
Commit: 833b34776a6afe08f34d035318c88dffe7f9cb31
Parents: d0c9e78
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Oct 2 17:40:31 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/WindowCrossJoinTest.java      |   6 ++
 .../api/operators/co/SelfConnectionTest.java    |   5 +
 .../join/WindowJoinITCase.java                  | 101 ++++++++++---------
 .../join/WindowJoinITCase.java                  | 101 ++++++++++---------
 4 files changed, 113 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/833b3477/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
index 0137682..7d2a131 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
@@ -33,6 +33,7 @@ import org.apache.flink.streaming.api.windowing.helper.Timestamp;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
@@ -50,6 +51,11 @@ public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
+	/**
+	 * TODO: enable once new join operator is ready
+	 * @throws Exception
+	 */
+	@Ignore
 	@Test
 	public void test() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

http://git-wip-us.apache.org/repos/asf/flink/blob/833b3477/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
index 19a61ba..fc9de1d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
@@ -39,13 +39,18 @@ import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
 
 	private static List<String> expected;
 
+	/**
+	 * TODO: enable once new join operator is implemented
+	 */
 	@SuppressWarnings({ "unchecked", "rawtypes" })
+	@Ignore
 	@Test
 	public void sameDataStreamTest() {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/833b3477/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
index e657b67..aae4b93 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
@@ -1,50 +1,51 @@
-/*
- * 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.test.exampleJavaPrograms.join;
-
-import org.apache.flink.streaming.examples.join.WindowJoin;
-import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class WindowJoinITCase extends StreamingProgramTestBase {
-
-	protected String gradesPath;
-	protected String salariesPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// since the two sides of the join might have different speed
-		// the exact output can not be checked just whether it is well-formed
-		// checks that the result lines look like e.g. (bob, 2, 2015)
-		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-	}
-}
+///*
+// * 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.
+// */
+//
+// TODO: reactivate once we have new join implementation
+//package org.apache.flink.streaming.test.exampleJavaPrograms.join;
+//
+//import org.apache.flink.streaming.examples.join.WindowJoin;
+//import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+//import org.apache.flink.streaming.util.StreamingProgramTestBase;
+//
+//public class WindowJoinITCase extends StreamingProgramTestBase {
+//
+//	protected String gradesPath;
+//	protected String salariesPath;
+//	protected String resultPath;
+//
+//	@Override
+//	protected void preSubmit() throws Exception {
+//		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+//		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+//		resultPath = getTempDirPath("result");
+//	}
+//
+//	@Override
+//	protected void postSubmit() throws Exception {
+//		// since the two sides of the join might have different speed
+//		// the exact output can not be checked just whether it is well-formed
+//		// checks that the result lines look like e.g. (bob, 2, 2015)
+//		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
+//	}
+//
+//	@Override
+//	protected void testProgram() throws Exception {
+//		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+//	}
+//}

http://git-wip-us.apache.org/repos/asf/flink/blob/833b3477/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
index 08ce890..0aa884f 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
@@ -1,50 +1,51 @@
-/*
- * 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.test.exampleScalaPrograms.join;
-
-import org.apache.flink.streaming.scala.examples.join.WindowJoin;
-import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class WindowJoinITCase extends StreamingProgramTestBase {
-
-	protected String gradesPath;
-	protected String salariesPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// since the two sides of the join might have different speed
-		// the exact output can not be checked just whether it is well-formed
-		// checks that the result lines look like e.g. Person(bob, 2, 2015)
-		checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-	}
-}
+///*
+// * 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.
+// */
+//
+// TODO: reactivate once we have new join implementation
+//package org.apache.flink.streaming.test.exampleScalaPrograms.join;
+//
+//import org.apache.flink.streaming.scala.examples.join.WindowJoin;
+//import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+//import org.apache.flink.streaming.util.StreamingProgramTestBase;
+//
+//public class WindowJoinITCase extends StreamingProgramTestBase {
+//
+//	protected String gradesPath;
+//	protected String salariesPath;
+//	protected String resultPath;
+//
+//	@Override
+//	protected void preSubmit() throws Exception {
+//		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+//		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+//		resultPath = getTempDirPath("result");
+//	}
+//
+//	@Override
+//	protected void postSubmit() throws Exception {
+//		// since the two sides of the join might have different speed
+//		// the exact output can not be checked just whether it is well-formed
+//		// checks that the result lines look like e.g. Person(bob, 2, 2015)
+//		checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
+//	}
+//
+//	@Override
+//	protected void testProgram() throws Exception {
+//		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+//	}
+//}


[04/13] flink git commit: [FLINK-2550] Simplify Stream Java API Class Names

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichAllWindowFunction.java
new file mode 100644
index 0000000..d78e2c3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichAllWindowFunction.java
@@ -0,0 +1,25 @@
+/**
+ * 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.functions.windowing;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+public abstract class RichAllWindowFunction<IN, OUT, W extends Window> extends AbstractRichFunction implements AllWindowFunction<IN, OUT, W> {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichKeyedWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichKeyedWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichKeyedWindowFunction.java
deleted file mode 100644
index 90ccb40..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichKeyedWindowFunction.java
+++ /dev/null
@@ -1,25 +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.functions.windowing;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-public abstract class RichKeyedWindowFunction<IN, OUT, KEY, W extends Window> extends AbstractRichFunction implements KeyedWindowFunction<IN, OUT, KEY, W> {
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
index b40b74a..0d40bbd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
@@ -20,6 +20,6 @@ package org.apache.flink.streaming.api.functions.windowing;
 import org.apache.flink.api.common.functions.AbstractRichFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
-public abstract class RichWindowFunction<IN, OUT, W extends Window> extends AbstractRichFunction implements WindowFunction<IN, OUT, W> {
+public abstract class RichWindowFunction<IN, OUT, KEY, W extends Window> extends AbstractRichFunction implements WindowFunction<IN, OUT, KEY, W> {
 	private static final long serialVersionUID = 1L;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
index 1a4304e..eda12c0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
@@ -25,19 +25,23 @@ import org.apache.flink.util.Collector;
 import java.io.Serializable;
 
 /**
- * Base interface for functions that are evaluated over non-keyed windows.
+ * Base interface for functions that are evaluated over keyed (grouped) windows.
  *
  * @param <IN> The type of the input value.
  * @param <OUT> The type of the output value.
+ * @param <KEY> The type of the key.
  */
-public interface WindowFunction<IN, OUT,  W extends Window> extends Function, Serializable {
+public interface WindowFunction<IN, OUT, KEY, W extends Window> extends Function, Serializable {
 
 	/**
-	 * 
-	 * @param values
-	 * @param out
+	 * Evaluates the window and outputs none or several elements.
+	 *
+	 * @param key The key for which this window is evaluated.
+	 * @param window The window that is being evaluated.
+	 * @param values The elements in the window being evaluated.
+	 * @param out A collector for emitting elements.
 	 * 
 	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery. 
 	 */
-	void evaluate(W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
+	void apply(KEY key, W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
index 496da6b..55c1be0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.runtime.util.UnionIterator;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
@@ -34,13 +34,13 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 
 	private final KeyMap.LazyFactory<ArrayList<Type>> listFactory = getListFactory();
 
-	private final KeyedWindowFunction<Type, Result, Key, Window> function;
+	private final WindowFunction<Type, Result, Key, Window> function;
 	
 	private long evaluationPass;
 
 	// ------------------------------------------------------------------------
 	
-	public AccumulatingKeyedTimePanes(KeySelector<Type, Key> keySelector, KeyedWindowFunction<Type, Result, Key, Window> function) {
+	public AccumulatingKeyedTimePanes(KeySelector<Type, Key> keySelector, WindowFunction<Type, Result, Key, Window> function) {
 		this.keySelector = keySelector;
 		this.function = function;
 	}
@@ -59,7 +59,7 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 		if (previousPanes.isEmpty()) {
 			// optimized path for single pane case (tumbling window)
 			for (KeyMap.Entry<Key, ArrayList<Type>> entry : latestPane) {
-				function.evaluate(entry.getKey(), window, entry.getValue(), out);
+				function.apply(entry.getKey(), window, entry.getValue(), out);
 			}
 		}
 		else {
@@ -77,7 +77,7 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 	
 	static final class WindowFunctionTraversal<Key, Type, Result> implements KeyMap.TraversalEvaluator<Key, ArrayList<Type>> {
 
-		private final KeyedWindowFunction<Type, Result, Key, Window> function;
+		private final WindowFunction<Type, Result, Key, Window> function;
 		
 		private final UnionIterator<Type> unionIterator;
 		
@@ -87,7 +87,7 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 
 		private TimeWindow window;
 
-		WindowFunctionTraversal(KeyedWindowFunction<Type, Result, Key, Window> function, TimeWindow window, Collector<Result> out) {
+		WindowFunctionTraversal(WindowFunction<Type, Result, Key, Window> function, TimeWindow window, Collector<Result> out) {
 			this.function = function;
 			this.out = out;
 			this.unionIterator = new UnionIterator<>();
@@ -108,7 +108,7 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 
 		@Override
 		public void keyDone() throws Exception {
-			function.evaluate(currentKey, window, unionIterator, out);
+			function.apply(currentKey, window, unionIterator, out);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
index ace3823..3bcffbc 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
@@ -20,19 +20,19 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
 
 public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT> 
-		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, KeyedWindowFunction<IN, OUT, KEY, TimeWindow>>  {
+		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, WindowFunction<IN, OUT, KEY, TimeWindow>>  {
 
 	private static final long serialVersionUID = 7305948082830843475L;
 
 	
 	public AccumulatingProcessingTimeWindowOperator(
-			KeyedWindowFunction<IN, OUT, KEY, TimeWindow> function,
+			WindowFunction<IN, OUT, KEY, TimeWindow> function,
 			KeySelector<IN, KEY> keySelector,
 			long windowLength,
 			long windowSlide)
@@ -43,7 +43,7 @@ public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT>
 	@Override
 	protected AccumulatingKeyedTimePanes<IN, KEY, OUT> createPanes(KeySelector<IN, KEY> keySelector, Function function) {
 		@SuppressWarnings("unchecked")
-		KeyedWindowFunction<IN, OUT, KEY, Window> windowFunction = (KeyedWindowFunction<IN, OUT, KEY, Window>) function;
+		WindowFunction<IN, OUT, KEY, Window> windowFunction = (WindowFunction<IN, OUT, KEY, Window>) function;
 		
 		return new AccumulatingKeyedTimePanes<>(keySelector, windowFunction);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
index d5ed6cb..53df838 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
 import org.apache.flink.streaming.api.windowing.evictors.Evictor;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
@@ -41,7 +41,7 @@ public class EvictingNonKeyedWindowOperator<IN, OUT, W extends Window> extends N
 
 	public EvictingNonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
 			WindowBufferFactory<? super IN, ? extends EvictingWindowBuffer<IN>> windowBufferFactory,
-			WindowFunction<IN, OUT, W> windowFunction,
+			AllWindowFunction<IN, OUT, W> windowFunction,
 			Trigger<? super IN, ? super W> trigger,
 			Evictor<? super IN, ? super W> evictor) {
 		super(windowAssigner, windowBufferFactory, windowFunction, trigger);
@@ -77,7 +77,7 @@ public class EvictingNonKeyedWindowOperator<IN, OUT, W extends Window> extends N
 
 		windowBuffer.removeElements(toEvict);
 
-		userFunction.evaluate(
+		userFunction.apply(
 				window,
 				bufferAndTrigger.f0.getUnpackedElements(),
 				timestampedCollector);

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index cddcc42..334eb54 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.EvictingWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuffer;
@@ -44,7 +44,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 	public EvictingWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
 			KeySelector<IN, K> keySelector,
 			WindowBufferFactory<? super IN, ? extends EvictingWindowBuffer<IN>> windowBufferFactory,
-			KeyedWindowFunction<IN, OUT, K, W> windowFunction,
+			WindowFunction<IN, OUT, K, W> windowFunction,
 			Trigger<? super IN, ? super W> trigger,
 			Evictor<? super IN, ? super W> evictor) {
 		super(windowAssigner, keySelector, windowBufferFactory, windowFunction, trigger);
@@ -87,7 +87,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 
 		windowBuffer.removeElements(toEvict);
 
-		userFunction.evaluate(key,
+		userFunction.apply(key,
 				window,
 				bufferAndTrigger.f0.getUnpackedElements(),
 				timestampedCollector);

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
index 3a85759..d48643d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
@@ -46,7 +46,7 @@ import java.util.Map;
 import java.util.Set;
 
 public class NonKeyedWindowOperator<IN, OUT, W extends Window>
-		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, W>>
+		extends AbstractUdfStreamOperator<OUT, AllWindowFunction<IN, OUT, W>>
 		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
 
 	private static final long serialVersionUID = 1L;
@@ -72,7 +72,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
 
 	public NonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
 			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
-			WindowFunction<IN, OUT, W> windowFunction,
+			AllWindowFunction<IN, OUT, W> windowFunction,
 			Trigger<? super IN, ? super W> trigger) {
 
 		super(windowFunction);
@@ -157,7 +157,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
 		}
 
 
-		userFunction.evaluate(
+		userFunction.apply(
 				window,
 				bufferAndTrigger.f0.getUnpackedElements(),
 				timestampedCollector);

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index cda4481..2d4635f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
@@ -47,7 +47,7 @@ import java.util.Map;
 import java.util.Set;
 
 public class WindowOperator<K, IN, OUT, W extends Window>
-		extends AbstractUdfStreamOperator<OUT, KeyedWindowFunction<IN, OUT, K, W>>
+		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, K, W>>
 		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
 
 	private static final long serialVersionUID = 1L;
@@ -75,7 +75,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 	public WindowOperator(WindowAssigner<? super IN, W> windowAssigner,
 			KeySelector<IN, K> keySelector,
 			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
-			KeyedWindowFunction<IN, OUT, K, W> windowFunction,
+			WindowFunction<IN, OUT, K, W> windowFunction,
 			Trigger<? super IN, ? super W> trigger) {
 
 		super(windowFunction);
@@ -181,7 +181,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 		}
 
 
-		userFunction.evaluate(key,
+		userFunction.apply(key,
 				window,
 				bufferAndTrigger.f0.getUnpackedElements(),
 				timestampedCollector);

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
index 0ac352b..a8c4b49 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
@@ -157,7 +157,7 @@ public class StreamingRuntimeContext extends RuntimeUDFContext {
 				return new PartitionedStreamOperatorState(provider, statePartitioner, getUserCodeClassLoader());
 			} else {
 				throw new RuntimeException(
-						"Partitioned state can only be used with KeyedDataStreams.");
+						"Partitioned state can only be used with KeyedStreams.");
 			}
 		} else {
 			return new StreamOperatorState(provider);

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
index 207b1b1..ead3af8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
@@ -46,7 +46,7 @@ import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.KeyedDataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.operators.Output;
@@ -113,7 +113,7 @@ public class StatefulOperatorTest extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(3);
 
-		KeyedDataStream<Integer, Integer> keyedStream = env
+		KeyedStream<Integer, Integer> keyedStream = env
 				.fromCollection(Arrays.asList(0, 1, 2, 3, 4, 5, 6))
 				.keyBy(new ModKey(4));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index 99a2e14..89672df 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
@@ -49,7 +49,7 @@ import static org.junit.Assert.*;
 public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 	@SuppressWarnings("unchecked")
-	private final KeyedWindowFunction<String, String, String, TimeWindow> mockFunction = mock(KeyedWindowFunction.class);
+	private final WindowFunction<String, String, String, TimeWindow> mockFunction = mock(WindowFunction.class);
 
 	@SuppressWarnings("unchecked")
 	private final KeySelector<String, String> mockKeySelector = mock(KeySelector.class);
@@ -61,11 +61,14 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 		}
 	};
 	
-	private final KeyedWindowFunction<Integer, Integer, Integer, TimeWindow> validatingIdentityFunction =
-			new KeyedWindowFunction<Integer, Integer, Integer, TimeWindow>()
+	private final WindowFunction<Integer, Integer, Integer, TimeWindow> validatingIdentityFunction =
+			new WindowFunction<Integer, Integer, Integer, TimeWindow>()
 	{
 		@Override
-		public void evaluate(Integer key, TimeWindow window, Iterable<Integer> values, Collector<Integer> out) {
+		public void apply(Integer key,
+				TimeWindow window,
+				Iterable<Integer> values,
+				Collector<Integer> out) {
 			for (Integer val : values) {
 				assertEquals(key, val);
 				out.collect(val);
@@ -472,7 +475,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
 			when(mockContext.getTaskName()).thenReturn("Test task name");
 
-			KeyedWindowFunction<Integer, Integer, Integer, TimeWindow> failingFunction = new FailingFunction(100);
+			WindowFunction<Integer, Integer, Integer, TimeWindow> failingFunction = new FailingFunction(100);
 
 			// the operator has a window time that is so long that it will not fire in this test
 			final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
@@ -523,7 +526,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 	// ------------------------------------------------------------------------
 	
-	private static class FailingFunction implements KeyedWindowFunction<Integer, Integer, Integer, TimeWindow> {
+	private static class FailingFunction implements WindowFunction<Integer, Integer, Integer, TimeWindow> {
 
 		private final int failAfterElements;
 		
@@ -534,7 +537,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 		}
 
 		@Override
-		public void evaluate(Integer integer, TimeWindow window, Iterable<Integer> values, Collector<Integer> out) throws Exception {
+		public void apply(Integer integer,
+				TimeWindow window,
+				Iterable<Integer> values,
+				Collector<Integer> out) throws Exception {
 			for (Integer i : values) {
 				out.collect(i);
 				numElements++;

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
new file mode 100644
index 0000000..ee8c6d6
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * These tests verify that the api calls on
+ * {@link org.apache.flink.streaming.api.datastream.AllWindowedStream} instantiate
+ * the correct window operator.
+ */
+public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
+
+	/**
+	 * These tests ensure that the fast aligned time windows operator is used if the
+	 * conditions are right.
+	 *
+	 * TODO: update once fast aligned time windows operator is in
+	 */
+	@Ignore
+	@Test
+	public void testFastTimeWindows() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void apply(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testNonEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.trigger(CountTrigger.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void apply(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.evictor(CountEvictor.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof EvictingNonKeyedWindowOperator);
+		EvictingNonKeyedWindowOperator winOperator1 = (EvictingNonKeyedWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.evictor(TimeEvictor.of(100))
+				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void apply(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof EvictingNonKeyedWindowOperator);
+		EvictingNonKeyedWindowOperator winOperator2 = (EvictingNonKeyedWindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class DummyReducer extends RichReduceFunction<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 value1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
index 0dfceab..3139941 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
@@ -23,8 +23,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
@@ -56,7 +55,7 @@ public class EvictingNonKeyedWindowOperatorTest {
 		EvictingNonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new EvictingNonKeyedWindowOperator<>(
 				GlobalWindows.create(),
 				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
-				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
+				new ReduceAllWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
 				CountTrigger.of(WINDOW_SLIDE),
 				CountEvictor.of(WINDOW_SIZE));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
index 01381f9..3d9605e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
 import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -56,7 +56,7 @@ public class EvictingWindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
-				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
+				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
 				CountTrigger.of(WINDOW_SLIDE),
 				CountEvictor.of(WINDOW_SIZE));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
index b74b3ea..9b0bcc4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
@@ -23,8 +23,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
@@ -76,7 +75,7 @@ public class NonKeyedWindowOperatorTest {
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
 				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
 				windowBufferFactory,
-				new ReduceWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -156,7 +155,7 @@ public class NonKeyedWindowOperatorTest {
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
 				TumblingTimeWindows.of(WINDOW_SIZE),
 				windowBufferFactory,
-				new ReduceWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -234,7 +233,7 @@ public class NonKeyedWindowOperatorTest {
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
 				GlobalWindows.create(),
 				windowBufferFactory,
-				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceAllWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -312,7 +311,7 @@ public class NonKeyedWindowOperatorTest {
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
 				GlobalWindows.create(),
 				windowBufferFactory,
-				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceAllWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
deleted file mode 100644
index 4babee1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
- * the correct window operator.
- */
-public class NonParallelWindowDataStreamTranslationTest extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 *
-	 * TODO: update once fast aligned time windows operator is in
-	 */
-	@Ignore
-	@Test
-	public void testFastTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
-				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testNonEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
-				.trigger(CountTrigger.of(100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
-		NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(1000))
-				.trigger(CountTrigger.of(100))
-				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
-		NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
-				.evictor(CountEvictor.of(100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof EvictingNonKeyedWindowOperator);
-		EvictingNonKeyedWindowOperator winOperator1 = (EvictingNonKeyedWindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
-		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(1000))
-				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(100))
-				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof EvictingNonKeyedWindowOperator);
-		EvictingNonKeyedWindowOperator winOperator2 = (EvictingNonKeyedWindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
-		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	// ------------------------------------------------------------------------
-	//  UDFs
-	// ------------------------------------------------------------------------
-
-	public static class DummyReducer extends RichReduceFunction<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 value1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
index aaf21e0..76d7bfe 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
@@ -21,9 +21,10 @@ import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.WindowedStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.windowing.time.Time;
@@ -38,7 +39,7 @@ import java.util.concurrent.TimeUnit;
 
 /**
  * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
+ * {@link WindowedStream} instantiate
  * the correct window operator.
  */
 public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase {
@@ -67,11 +68,11 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
 				.timeWindow(Time.of(1000, TimeUnit.MILLISECONDS))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
-					public void evaluate(Tuple tuple,
+					public void apply(Tuple tuple,
 							TimeWindow window,
 							Iterable<Tuple2<String, Integer>> values,
 							Collector<Tuple2<String, Integer>> out) throws Exception {
@@ -110,11 +111,11 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS))
-				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
-					public void evaluate(
+					public void apply(
 							TimeWindow window,
 							Iterable<Tuple2<String, Integer>> values,
 							Collector<Tuple2<String, Integer>> out) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index 3107d51..1bfd1d5 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
-import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
 import org.apache.flink.streaming.api.windowing.triggers.ContinuousWatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
@@ -76,7 +76,7 @@ public class WindowOperatorTest {
 				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceKeyedWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -163,7 +163,7 @@ public class WindowOperatorTest {
 				TumblingTimeWindows.of(WINDOW_SIZE),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceKeyedWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -246,7 +246,7 @@ public class WindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -331,7 +331,7 @@ public class WindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index 43e7715..a3e6085 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -21,8 +21,9 @@ import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.WindowedStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
@@ -41,7 +42,7 @@ import org.junit.Test;
 
 /**
  * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
+ * {@link WindowedStream} instantiate
  * the correct window operator.
  */
 public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
@@ -70,11 +71,11 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
 				.window(SlidingProcessingTimeWindows.of(1000, 100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
-					public void evaluate(Tuple tuple,
+					public void apply(Tuple tuple,
 							TimeWindow window,
 							Iterable<Tuple2<String, Integer>> values,
 							Collector<Tuple2<String, Integer>> out) throws Exception {
@@ -114,11 +115,11 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 				.keyBy(0)
 				.window(TumblingProcessingTimeWindows.of(1000))
 				.trigger(CountTrigger.of(100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
-					public void evaluate(Tuple tuple,
+					public void apply(Tuple tuple,
 							TimeWindow window,
 							Iterable<Tuple2<String, Integer>> values,
 							Collector<Tuple2<String, Integer>> out) throws Exception {
@@ -164,11 +165,11 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 				.window(TumblingProcessingTimeWindows.of(1000))
 				.trigger(CountTrigger.of(100))
 				.evictor(TimeEvictor.of(100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
-					public void evaluate(Tuple tuple,
+					public void apply(Tuple tuple,
 							TimeWindow window,
 							Iterable<Tuple2<String, Integer>> values,
 							Collector<Tuple2<String, Integer>> out) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
index a5f1e89..5d32b8e 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
@@ -26,7 +26,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
@@ -81,10 +81,10 @@ public class GroupedProcessingTimeWindowExample {
 			.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
 			.reduceWindow(new SummingReducer())
 
-			// alternative: use a mapWindow function which does not pre-aggregate
+			// alternative: use a apply function which does not pre-aggregate
 //			.keyBy(new FirstFieldKeyExtractor<Tuple2<Long, Long>, Long>())
 //			.window(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
-//			.mapWindow(new SummingWindowFunction())
+//			.apply(new SummingWindowFunction())
 				
 			.addSink(new SinkFunction<Tuple2<Long, Long>>() {
 				@Override
@@ -104,10 +104,10 @@ public class GroupedProcessingTimeWindowExample {
 		}
 	}
 
-	public static class SummingWindowFunction implements KeyedWindowFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long, Window> {
+	public static class SummingWindowFunction implements WindowFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long, Window> {
 
 		@Override
-		public void evaluate(Long key, Window window, Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
+		public void apply(Long key, Window window, Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
 			long sum = 0L;
 			for (Tuple2<Long, Long> value : values) {
 				sum += value.f1;

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 9f4f52a..ca5fc48 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -39,6 +39,7 @@ import org.apache.flink.streaming.util.serialization.SerializationSchema
 import org.apache.flink.util.Collector
 import org.apache.flink.api.common.functions.{RichMapFunction, RichFlatMapFunction, RichFilterFunction}
 import org.apache.flink.streaming.api.scala.function.StatefulFunction
+import org.apache.flink.streaming.api.datastream.{KeyedStream => JavaKeyedStream}
 
 class DataStream[T](javaStream: JavaStream[T]) {
 
@@ -246,7 +247,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * Groups the elements of a DataStream by the given field expressions to
    * be used with grouped operators like grouped reduce or grouped aggregations.
    */
-  def groupBy(firstField: String, otherFields: String*): GroupedDataStream[T, JavaTuple] = 
+  def groupBy(firstField: String, otherFields: String*): GroupedDataStream[T, JavaTuple] =
    javaStream.groupBy(firstField +: otherFields.toArray: _*)   
   
   /**
@@ -601,7 +602,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
   }
 
   private[flink] def isStatePartitioned: Boolean = {
-    javaStream.isInstanceOf[KeyedDataStream[_, _]]
+    javaStream.isInstanceOf[JavaKeyedStream[_, _]]
   }
 
   /**


[06/13] flink git commit: [FLINK-2550] Rename ConnectedDataStream to ConnectedStreams, Remove some operations

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java
deleted file mode 100644
index dc6ea34..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamReduce.java
+++ /dev/null
@@ -1,86 +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.operators.co;
-
-import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class CoStreamReduce<IN1, IN2, OUT>
-		extends AbstractUdfStreamOperator<OUT, CoReduceFunction<IN1, IN2, OUT>>
-		implements TwoInputStreamOperator<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected IN1 currentValue1 = null;
-	protected IN2 currentValue2 = null;
-
-	// We keep track of watermarks from both inputs, the combined input is the minimum
-	// Once the minimum advances we emit a new watermark for downstream operators
-	private long combinedWatermark = Long.MIN_VALUE;
-	private long input1Watermark = Long.MIN_VALUE;
-	private long input2Watermark = Long.MIN_VALUE;
-
-	public CoStreamReduce(CoReduceFunction<IN1, IN2, OUT> coReducer) {
-		super(coReducer);
-		currentValue1 = null;
-		currentValue2 = null;
-	}
-
-	@Override
-	public void processElement1(StreamRecord<IN1> element) throws Exception {
-		if (currentValue1 != null) {
-			currentValue1 = userFunction.reduce1(currentValue1, element.getValue());
-		} else {
-			currentValue1 = element.getValue();
-		}
-		output.collect(element.replace(userFunction.map1(currentValue1)));
-	}
-
-	@Override
-	public void processElement2(StreamRecord<IN2> element) throws Exception {
-		if (currentValue2 != null) {
-			currentValue2 = userFunction.reduce2(currentValue2, element.getValue());
-		} else {
-			currentValue2 = element.getValue();
-		}
-		output.collect(element.replace(userFunction.map2(currentValue2)));
-	}
-
-	@Override
-	public void processWatermark1(Watermark mark) throws Exception {
-		input1Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	@Override
-	public void processWatermark2(Watermark mark) throws Exception {
-		input2Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java
deleted file mode 100644
index 4bfe2ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamWindow.java
+++ /dev/null
@@ -1,228 +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.operators.co;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.math.util.MathUtils;
-import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.state.CircularFifoList;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class CoStreamWindow<IN1, IN2, OUT>
-		extends AbstractUdfStreamOperator<OUT, CoWindowFunction<IN1, IN2, OUT>>
-		implements TwoInputStreamOperator<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected long windowSize;
-	protected long slideSize;
-	protected CircularFifoList<StreamRecord<IN1>> circularList1;
-	protected CircularFifoList<StreamRecord<IN2>> circularList2;
-	protected TimestampWrapper<IN1> timeStamp1;
-	protected TimestampWrapper<IN2> timeStamp2;
-
-	protected StreamWindow window;
-
-	protected long startTime;
-	protected long nextRecordTime;
-
-	// We keep track of watermarks from both inputs, the combined input is the minimum
-	// Once the minimum advances we emit a new watermark for downstream operators
-	private long combinedWatermark = Long.MIN_VALUE;
-	private long input1Watermark = Long.MIN_VALUE;
-	private long input2Watermark = Long.MIN_VALUE;
-
-	public CoStreamWindow(CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize,
-			long slideInterval, TimestampWrapper<IN1> timeStamp1, TimestampWrapper<IN2> timeStamp2) {
-		super(coWindowFunction);
-		this.windowSize = windowSize;
-		this.slideSize = slideInterval;
-		this.circularList1 = new CircularFifoList<StreamRecord<IN1>>();
-		this.circularList2 = new CircularFifoList<StreamRecord<IN2>>();
-		this.timeStamp1 = timeStamp1;
-		this.timeStamp2 = timeStamp2;
-		this.startTime = timeStamp1.getStartTime();
-
-		this.window = new StreamWindow();
-	}
-
-	@Override
-	public void processElement1(StreamRecord<IN1> element) throws Exception {
-		window.addToBuffer1(element.getValue());
-	}
-
-	@Override
-	public void processElement2(StreamRecord<IN2> element) throws Exception {
-		window.addToBuffer2(element.getValue());
-	}
-
-	@SuppressWarnings("unchecked")
-	protected void callUserFunction() throws Exception {
-
-		List<IN1> first = new ArrayList<IN1>();
-		List<IN2> second = new ArrayList<IN2>();
-
-		// TODO: Give operators a way to copy elements
-
-		for (IN1 element : window.circularList1.getElements()) {
-			first.add(element);
-		}
-		for (IN2 element : window.circularList2.getElements()) {
-			second.add(element);
-		}
-
-		TimestampedCollector<OUT> timestampedCollector = new TimestampedCollector<OUT>(output);
-		timestampedCollector.setTimestamp(System.currentTimeMillis());
-		if (!window.circularList1.isEmpty() || !window.circularList2.isEmpty()) {
-			userFunction.coWindow(first, second, timestampedCollector);
-		}
-	}
-
-	@Override
-	public void processWatermark1(Watermark mark) throws Exception {
-		input1Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	@Override
-	public void processWatermark2(Watermark mark) throws Exception {
-		input2Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	protected class StreamWindow implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		protected int granularity;
-		protected int batchPerSlide;
-		protected long numberOfBatches;
-
-		protected long minibatchCounter;
-
-		protected CircularFifoList<IN1> circularList1;
-		protected CircularFifoList<IN2> circularList2;
-
-		public StreamWindow() {
-			this.granularity = (int) MathUtils.gcd(windowSize, slideSize);
-			this.batchPerSlide = (int) (slideSize / granularity);
-			this.numberOfBatches = windowSize / granularity;
-			this.circularList1 = new CircularFifoList<IN1>();
-			this.circularList2 = new CircularFifoList<IN2>();
-			this.minibatchCounter = 0;
-		}
-
-		public void addToBuffer1(IN1 nextValue) throws Exception {
-			checkWindowEnd(timeStamp1.getTimestamp(nextValue));
-			if (minibatchCounter >= 0) {
-				circularList1.add(nextValue);
-			}
-		}
-
-		public void addToBuffer2(IN2 nextValue) throws Exception {
-			checkWindowEnd(timeStamp2.getTimestamp(nextValue));
-			if (minibatchCounter >= 0) {
-				circularList2.add(nextValue);
-			}
-		}
-
-		protected synchronized void checkWindowEnd(long timeStamp) throws Exception{
-			nextRecordTime = timeStamp;
-
-			while (miniBatchEnd()) {
-				circularList1.newSlide();
-				circularList2.newSlide();
-				minibatchCounter++;
-				if (windowEnd()) {
-					callUserFunction();
-					circularList1.shiftWindow(batchPerSlide);
-					circularList2.shiftWindow(batchPerSlide);
-				}
-			}
-		}
-
-		protected boolean miniBatchEnd() {
-			if (nextRecordTime < startTime + granularity) {
-				return false;
-			} else {
-				startTime += granularity;
-				return true;
-			}
-		}
-
-		public boolean windowEnd() {
-			if (minibatchCounter == numberOfBatches) {
-				minibatchCounter -= batchPerSlide;
-				return true;
-			}
-			return false;
-		}
-
-		public void reduceLastBatch() throws Exception{
-			if (!miniBatchEnd()) {
-				callUserFunction();
-			}
-		}
-
-		public Iterable<IN1> getIterable1() {
-			return circularList1.getIterable();
-		}
-
-		public Iterable<IN2> getIterable2() {
-			return circularList2.getIterable();
-		}
-
-		@Override
-		public String toString() {
-			return circularList1.toString();
-		}
-
-	}
-
-	@Override
-	public void close() throws Exception {
-		if (!window.miniBatchEnd()) {
-			try {
-				callUserFunction();
-			} catch (Exception e) {
-				throw new RuntimeException("Could not call user function in CoStreamWindow.close()", e);
-			}
-		}
-		super.close();
-	}
-
-	public void setSlideSize(long slideSize) {
-		this.slideSize = slideSize;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 9775392..337d97b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.ConnectedDataStream;
+import org.apache.flink.streaming.api.datastream.ConnectedStreams;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
@@ -62,7 +62,6 @@ import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.NoOpSink;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
@@ -127,7 +126,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
 	/**
 	 * Tests that {@link DataStream#groupBy} and {@link DataStream#partitionByHash} result in
-	 * different and correct topologies. Does the some for the {@link ConnectedDataStream}.
+	 * different and correct topologies. Does the some for the {@link ConnectedStreams}.
 	 */
 	@Test
 	@SuppressWarnings("unchecked")
@@ -136,7 +135,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream src1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
 		DataStream src2 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
-		ConnectedDataStream connected = src1.connect(src2);
+		ConnectedStreams connected = src1.connect(src2);
 
 		//Testing DataStream grouping
 		DataStream group1 = src1.groupBy(0);
@@ -204,20 +203,20 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertFalse(isGrouped(customPartition3));
 		assertFalse(isGrouped(customPartition4));
 
-		//Testing ConnectedDataStream grouping
-		ConnectedDataStream connectedGroup1 = connected.groupBy(0, 0);
+		//Testing ConnectedStreams grouping
+		ConnectedStreams connectedGroup1 = connected.groupBy(0, 0);
 		Integer downStreamId1 = createDownStreamId(connectedGroup1);
 
-		ConnectedDataStream connectedGroup2 = connected.groupBy(new int[]{0}, new int[]{0});
+		ConnectedStreams connectedGroup2 = connected.groupBy(new int[]{0}, new int[]{0});
 		Integer downStreamId2 = createDownStreamId(connectedGroup2);
 
-		ConnectedDataStream connectedGroup3 = connected.groupBy("f0", "f0");
+		ConnectedStreams connectedGroup3 = connected.groupBy("f0", "f0");
 		Integer downStreamId3 = createDownStreamId(connectedGroup3);
 
-		ConnectedDataStream connectedGroup4 = connected.groupBy(new String[]{"f0"}, new String[]{"f0"});
+		ConnectedStreams connectedGroup4 = connected.groupBy(new String[]{"f0"}, new String[]{"f0"});
 		Integer downStreamId4 = createDownStreamId(connectedGroup4);
 
-		ConnectedDataStream connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector());
+		ConnectedStreams connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector());
 		Integer downStreamId5 = createDownStreamId(connectedGroup5);
 
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1)));
@@ -241,20 +240,20 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isGrouped(connectedGroup4));
 		assertTrue(isGrouped(connectedGroup5));
 
-		//Testing ConnectedDataStream partitioning
-		ConnectedDataStream connectedPartition1 = connected.partitionByHash(0, 0);
+		//Testing ConnectedStreams partitioning
+		ConnectedStreams connectedPartition1 = connected.partitionByHash(0, 0);
 		Integer connectDownStreamId1 = createDownStreamId(connectedPartition1);
 
-		ConnectedDataStream connectedPartition2 = connected.partitionByHash(new int[]{0}, new int[]{0});
+		ConnectedStreams connectedPartition2 = connected.partitionByHash(new int[]{0}, new int[]{0});
 		Integer connectDownStreamId2 = createDownStreamId(connectedPartition2);
 
-		ConnectedDataStream connectedPartition3 = connected.partitionByHash("f0", "f0");
+		ConnectedStreams connectedPartition3 = connected.partitionByHash("f0", "f0");
 		Integer connectDownStreamId3 = createDownStreamId(connectedPartition3);
 
-		ConnectedDataStream connectedPartition4 = connected.partitionByHash(new String[]{"f0"}, new String[]{"f0"});
+		ConnectedStreams connectedPartition4 = connected.partitionByHash(new String[]{"f0"}, new String[]{"f0"});
 		Integer connectDownStreamId4 = createDownStreamId(connectedPartition4);
 
-		ConnectedDataStream connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
+		ConnectedStreams connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
 		Integer connectDownStreamId5 = createDownStreamId(connectedPartition5);
 
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
@@ -470,7 +469,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		StreamEdge splitEdge = env.getStreamGraph().getStreamEdge(unionFilter.getId(), sink.getTransformation().getId());
 		assertEquals("a", splitEdge.getSelectedNames().get(0));
 
-		ConnectedDataStream<Integer, Integer> connect = map.connect(flatMap);
+		ConnectedStreams<Integer, Integer> connect = map.connect(flatMap);
 		CoMapFunction<Integer, Integer, String> coMapper = new CoMapFunction<Integer, Integer, String>() {
 			@Override
 			public String map1(Integer value) {
@@ -606,7 +605,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return dataStream instanceof GroupedDataStream;
 	}
 
-	private static Integer createDownStreamId(ConnectedDataStream dataStream) {
+	private static Integer createDownStreamId(ConnectedStreams dataStream) {
 		SingleOutputStreamOperator coMap = dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
 			@Override
 			public Object map1(Tuple2<Long, Long> value) {
@@ -622,8 +621,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return coMap.getId();
 	}
 
-	private static boolean isGrouped(ConnectedDataStream dataStream) {
-		return (dataStream.getFirst() instanceof GroupedDataStream && dataStream.getSecond() instanceof GroupedDataStream);
+	private static boolean isGrouped(ConnectedStreams dataStream) {
+		return (dataStream.getFirstInput() instanceof GroupedDataStream && dataStream.getSecondInput() instanceof GroupedDataStream);
 	}
 
 	private static boolean isPartitioned(StreamEdge edge) {

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 285ee57..774f58d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -22,6 +22,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -31,7 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.IterativeDataStream;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream.ConnectedIterativeDataStream;
+import org.apache.flink.streaming.api.datastream.IterativeDataStream.ConnectedIterativeDataStreams;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -49,7 +50,6 @@ import org.apache.flink.streaming.util.EvenOddOutputSelector;
 import org.apache.flink.streaming.util.NoOpIntMap;
 import org.apache.flink.streaming.util.ReceiveCheckNoOpSink;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
@@ -112,7 +112,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		ConnectedIterativeDataStream<Integer, Integer> coIter = source.iterate().withFeedbackType(
+		ConnectedIterativeDataStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
 
@@ -151,7 +151,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
 		IterativeDataStream<Integer> iter1 = source.iterate();
-		ConnectedIterativeDataStream<Integer, Integer> coIter = source.iterate().withFeedbackType(
+		ConnectedIterativeDataStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
 
@@ -181,7 +181,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 
 		IterativeDataStream<Integer> iter1 = source.iterate();
 		// Calling withFeedbackType should create a new iteration
-		ConnectedIterativeDataStream<Integer, String> iter2 = iter1.withFeedbackType(String.class);
+		ConnectedIterativeDataStreams<Integer, String> iter2 = iter1.withFeedbackType(String.class);
 
 		iter1.closeWith(iter1.map(NoOpIntMap)).print();
 		iter2.closeWith(iter2.map(NoOpCoMap)).print();
@@ -395,7 +395,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 				.map(NoOpStrMap).name("ParallelizeMap");
 
 
-		ConnectedIterativeDataStream<Integer, String> coIt = env.fromElements(0, 0)
+		ConnectedIterativeDataStreams<Integer, String> coIt = env.fromElements(0, 0)
 				.map(NoOpIntMap).name("ParallelizeMap")
 				.iterate(2000)
 				.withFeedbackType("String");
@@ -403,7 +403,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		try {
 			coIt.groupBy(1, 2);
 			fail();
-		} catch (UnsupportedOperationException e) {
+		} catch (InvalidProgramException e) {
 			// this is expected
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
index 0989128..4c0f59f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.api;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.util.List;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.MapFunction;
@@ -30,8 +29,6 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
-import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
@@ -72,12 +69,6 @@ public class TypeFillTest extends StreamingMultipleProgramsTestBase {
 			fail();
 		} catch (Exception e) {
 		}
-		try {
-			source.connect(source).windowReduce(new TestCoWindow<Long, Long, String>(), 10, 100)
-					.print();
-			fail();
-		} catch (Exception e) {
-		}
 
 		env.addSource(new TestSource<Integer>()).returns("Integer");
 		source.map(new TestMap<Long, Long>()).returns(Long.class).print();
@@ -86,9 +77,6 @@ public class TypeFillTest extends StreamingMultipleProgramsTestBase {
 		source.connect(source).flatMap(new TestCoFlatMap<Long, Long, Integer>())
 				.returns(BasicTypeInfo.INT_TYPE_INFO).print();
 		
-		source.connect(source).windowReduce(new TestCoWindow<Long, Long, String>(), 10, 100)
-				.returns("String").print();
-
 		assertEquals(BasicTypeInfo.LONG_TYPE_INFO,
 				source.map(new TestMap<Long, Long>()).returns(Long.class).getType());
 
@@ -161,38 +149,4 @@ public class TypeFillTest extends StreamingMultipleProgramsTestBase {
 		}
 
 	}
-
-	private class TestCoReduce<IN1, IN2, OUT> implements CoReduceFunction<IN1, IN2, OUT> {
-
-		@Override
-		public IN1 reduce1(IN1 value1, IN1 value2) {
-			return null;
-		}
-
-		@Override
-		public IN2 reduce2(IN2 value1, IN2 value2) {
-			return null;
-		}
-
-		@Override
-		public OUT map1(IN1 value) {
-			return null;
-		}
-
-		@Override
-		public OUT map2(IN2 value) {
-			return null;
-		}
-
-	}
-
-	private class TestCoWindow<IN1, IN2, OUT> implements CoWindowFunction<IN1, IN2, OUT> {
-
-		@Override
-		public void coWindow(List<IN1> first, List<IN2> second, Collector<OUT> out)
-				throws Exception {
-		}
-
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
index 508f1a2..0137682 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
@@ -115,23 +115,6 @@ public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
 				.map(new ResultMap())
 				.addSink(joinResultSink);
 
-		inStream1
-				.cross(inStream2)
-				.onWindow(1000, new MyTimestamp<Tuple2<Integer, String>>(),
-						new MyTimestamp<Tuple1<Integer>>(), 100)
-				.with(new CrossFunction<Tuple2<Integer, String>, Tuple1<Integer>, Tuple2<Tuple2<Integer, String>, Tuple1<Integer>>>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Tuple2<Tuple2<Integer, String>, Tuple1<Integer>> cross(
-							Tuple2<Integer, String> val1, Tuple1<Integer> val2) throws Exception {
-						return new Tuple2<Tuple2<Integer, String>, Tuple1<Integer>>(val1, val2);
-					}
-				})
-				.map(new ResultMap())
-				.addSink(crossResultSink);
-
 		env.execute();
 
 		assertEquals(new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(joinExpectedResults),

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
index 3b05274..2246ffd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.graph;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.datastream.ConnectedDataStream;
+import org.apache.flink.streaming.api.datastream.ConnectedStreams;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -220,7 +220,7 @@ public class StreamGraphGeneratorTest extends StreamingMultipleProgramsTestBase
 		DataStream<Integer> source1 = env.fromElements(1, 10);
 		DataStream<Integer> source2 = env.fromElements(2, 11);
 
-		ConnectedDataStream<Integer, Integer> connectedSource = source1.connect(source2);
+		ConnectedStreams<Integer, Integer> connectedSource = source1.connect(source2);
 
 		OutputTypeConfigurableOperationWithTwoInputs outputTypeConfigurableOperation = new OutputTypeConfigurableOperationWithTwoInputs();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedDataStream.scala
deleted file mode 100644
index a1e9f74..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedDataStream.scala
+++ /dev/null
@@ -1,510 +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.scala
-
-import java.util
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.datastream.{ConnectedDataStream => JavaCStream, DataStream => JavaStream}
-import org.apache.flink.streaming.api.functions.co.{CoFlatMapFunction, CoMapFunction, CoReduceFunction, CoWindowFunction}
-import org.apache.flink.util.Collector
-import scala.collection.JavaConverters._
-import scala.reflect.ClassTag
-
-class ConnectedDataStream[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
-
-  /**
-   * Applies a CoMap transformation on a {@link ConnectedDataStream} and maps
-   * the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input and
-   * @param fun2 for each element of the second input. Each
-   * CoMapFunction call returns exactly one element.
-   *
-   * The CoMapFunction used to jointly transform the two input
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def map[R: TypeInformation: ClassTag](fun1: IN1 => R, fun2: IN2 => R): 
-  DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val comapper = new CoMapFunction[IN1, IN2, R] {
-      def map1(in1: IN1): R = cleanFun1(in1)
-      def map2(in2: IN2): R = cleanFun2(in2)
-    }
-
-    map(comapper)
-  }
-
-  /**
-   * 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
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def map[R: TypeInformation: ClassTag](coMapper: CoMapFunction[IN1, IN2, R]): 
-  DataStream[R] = {
-    if (coMapper == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    javaStream.map(coMapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
-   * maps the output to a common type. The transformation calls a
-   * {@link CoFlatMapFunction#flatMap1} for each element of the first input
-   * and {@link CoFlatMapFunction#flatMap2} 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
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](coFlatMapper: CoFlatMapFunction[IN1, IN2, R]): 
-  DataStream[R] = {
-    if (coFlatMapper == null) {
-      throw new NullPointerException("FlatMap function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    javaStream.flatMap(coFlatMapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
-   * maps the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input
-   * and @param fun2 for each element of the second
-   * input. Each CoFlatMapFunction call returns any number of elements
-   * including none.
-   *
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun1: (IN1, Collector[R]) => Unit, 
-      fun2: (IN2, Collector[R]) => Unit): DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("FlatMap functions must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
-      def flatMap1(value: IN1, out: Collector[R]): Unit = cleanFun1(value, out)
-      def flatMap2(value: IN2, out: Collector[R]): Unit = cleanFun2(value, out)
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
-   * maps the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input
-   * and @param fun2 for each element of the second
-   * input. Each CoFlatMapFunction call returns any number of elements
-   * including none.
-   *
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun1: IN1 => TraversableOnce[R],
-      fun2: IN2 => TraversableOnce[R]): DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("FlatMap functions must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
-      def flatMap1(value: IN1, out: Collector[R]) = { cleanFun1(value) foreach out.collect }
-      def flatMap2(value: IN2, out: Collector[R]) = { cleanFun2(value) foreach out.collect }
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * 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
-   * {@link ConnectedDataStream#reduce}
-   *
-   * @param keyPosition1
-   * The field used to compute the hashcode of the elements in the
-   * first input stream.
-   * @param keyPosition2
-   * The field used to compute the hashcode of the elements in the
-   * second input stream.
-   * @return @return The transformed { @link ConnectedDataStream}
-   */
-  def groupBy(keyPosition1: Int, keyPosition2: Int): ConnectedDataStream[IN1, IN2] = {
-    javaStream.groupBy(keyPosition1, keyPosition2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream. Groups the elements of
-   * input1 and input2 according to keyPositions1 and keyPositions2. Used for
-   * applying function on grouped data streams for example
-   * {@link ConnectedDataStream#reduce}
-   *
-   * @param keyPositions1
-   * The fields used to group the first input stream.
-   * @param keyPositions2
-   * The fields used to group the second input stream.
-   * @return @return The transformed { @link ConnectedDataStream}
-   */
-  def groupBy(keyPositions1: Array[Int], keyPositions2: Array[Int]): 
-  ConnectedDataStream[IN1, IN2] = {
-    javaStream.groupBy(keyPositions1, keyPositions2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream using key expressions. Groups
-   * the elements of input1 and input2 according to field1 and field2. A field
-   * expression is either the name of a public field or a getter method with
-   * parentheses of the {@link DataStream}S underlying type. A dot can be used
-   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-   *
-   * @param field1
-   * The grouping expression for the first input
-   * @param field2
-   * The grouping expression for the second input
-   * @return The grouped { @link ConnectedDataStream}
-   */
-  def groupBy(field1: String, field2: String): ConnectedDataStream[IN1, IN2] = {
-    javaStream.groupBy(field1, field2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream using key expressions. Groups
-   * the elements of input1 and input2 according to fields1 and fields2. A
-   * field expression is either the name of a public field or a getter method
-   * with parentheses of the {@link DataStream}S underlying type. A dot can be
-   * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-   * .
-   *
-   * @param fields1
-   * The grouping expressions for the first input
-   * @param fields2
-   * The grouping expressions for the second input
-   * @return The grouped { @link ConnectedDataStream}
-   */
-  def groupBy(fields1: Array[String], fields2: Array[String]): 
-  ConnectedDataStream[IN1, IN2] = {
-    javaStream.groupBy(fields1, fields2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream. Groups the elements of
-   * input1 and input2 using fun1 and fun2. Used for applying
-   * function on grouped data streams for example
-   * {@link ConnectedDataStream#reduce}
-   *
-   * @param fun1
-   * The function used for grouping the first input
-   * @param fun2
-   * The function used for grouping the second input
-   * @return The grouped { @link ConnectedDataStream}
-   */
-  def groupBy[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
-  ConnectedDataStream[IN1, IN2] = {
-
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val keyExtractor1 = new KeySelector[IN1, K] {
-      def getKey(in: IN1) = cleanFun1(in)
-    }
-    val keyExtractor2 = new KeySelector[IN2, L] {
-      def getKey(in: IN2) = cleanFun2(in)
-    }
-
-    javaStream.groupBy(keyExtractor1, keyExtractor2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 according to keyPosition1 and keyPosition2.
-   *
-   * @param keyPosition1
-   * The field used to compute the hashcode of the elements in the
-   * first input stream.
-   * @param keyPosition2
-   * The field used to compute the hashcode of the elements in the
-   * second input stream.
-   * @return The transformed { @link ConnectedDataStream}
-   */
-  def partitionByHash(keyPosition1: Int, keyPosition2: Int): ConnectedDataStream[IN1, IN2] = {
-    javaStream.partitionByHash(keyPosition1, keyPosition2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 according to keyPositions1 and keyPositions2.
-   *
-   * @param keyPositions1
-   * The fields used to partition the first input stream.
-   * @param keyPositions2
-   * The fields used to partition the second input stream.
-   * @return The transformed { @link ConnectedDataStream}
-   */
-  def partitionByHash(keyPositions1: Array[Int], keyPositions2: Array[Int]):
-  ConnectedDataStream[IN1, IN2] = {
-    javaStream.partitionByHash(keyPositions1, keyPositions2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream using key expressions. Partitions
-   * the elements of input1 and input2 according to field1 and field2. A field
-   * expression is either the name of a public field or a getter method with
-   * parentheses of the {@link DataStream}S underlying type. A dot can be used
-   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-   *
-   * @param field1
-   * The partitioning expression for the first input
-   * @param field2
-   * The partitioning expression for the second input
-   * @return The grouped { @link ConnectedDataStream}
-   */
-  def partitionByHash(field1: String, field2: String): ConnectedDataStream[IN1, IN2] = {
-    javaStream.partitionByHash(field1, field2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream using key expressions. Partitions
-   * the elements of input1 and input2 according to fields1 and fields2.
-   *
-   * @param fields1
-   * The partitioning expressions for the first input
-   * @param fields2
-   * The partitioning expressions for the second input
-   * @return The partitioned { @link ConnectedDataStream}
-   */
-  def partitionByHash(fields1: Array[String], fields2: Array[String]):
-  ConnectedDataStream[IN1, IN2] = {
-    javaStream.partitionByHash(fields1, fields2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 using fun1 and fun2.
-   *
-   * @param fun1
-   * The function used for partitioning the first input
-   * @param fun2
-   * The function used for partitioning the second input
-   * @return The partitioned { @link ConnectedDataStream}
-   */
-  def partitionByHash[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
-  ConnectedDataStream[IN1, IN2] = {
-
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-
-    val keyExtractor1 = new KeySelector[IN1, K] {
-      def getKey(in: IN1) = cleanFun1(in)
-    }
-    val keyExtractor2 = new KeySelector[IN2, L] {
-      def getKey(in: IN2) = cleanFun2(in)
-    }
-
-    javaStream.partitionByHash(keyExtractor1, keyExtractor2)
-  }
-
-  /**
-   * Applies a reduce transformation on a {@link ConnectedDataStream} and maps
-   * the outputs to a common type. If the {@link ConnectedDataStream} is
-   * batched or windowed then the reduce transformation is applied on every
-   * sliding batch/window of the data stream. If the connected 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.
-   *
-   * @param coReducer
-   * The { @link CoReduceFunction} that will be called for every
-   *             element of the inputs.
-   * @return The transformed { @link DataStream}.
-   */
-  def reduce[R: TypeInformation: ClassTag](coReducer: CoReduceFunction[IN1, IN2, R]): 
-  DataStream[R] = {
-    if (coReducer == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    javaStream.reduce(coReducer).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a reduce transformation on a {@link ConnectedDataStream} and maps
-   * the outputs to a common type. If the {@link ConnectedDataStream} is
-   * batched or windowed then the reduce transformation is applied on every
-   * sliding batch/window of the data stream. If the connected 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.
-   *
-   * @return The transformed { @link DataStream}.
-   */
-  def reduce[R: TypeInformation: ClassTag](reducer1: (IN1, IN1) => IN1,
-      reducer2: (IN2, IN2) => IN2,mapper1: IN1 => R, mapper2: IN2 => R): DataStream[R] = {
-    if (mapper1 == null || mapper2 == null) {
-      throw new NullPointerException("Map functions must not be null.")
-    }
-    if (reducer1 == null || reducer2 == null) {
-      throw new NullPointerException("Reduce functions must not be null.")
-    }
-
-    val cleanReducer1 = clean(reducer1)
-    val cleanReducer2 = clean(reducer2)
-    val cleanMapper1 = clean(mapper1)
-    val cleanMapper2 = clean(mapper2)
-
-    val reducer = new CoReduceFunction[IN1, IN2, R] {
-      def reduce1(value1: IN1, value2: IN1): IN1 = cleanReducer1(value1, value2)
-      def reduce2(value1: IN2, value2: IN2): IN2 = cleanReducer2(value1, value2)
-      def map1(value: IN1): R = cleanMapper1(value)
-      def map2(value: IN2): R = cleanMapper2(value)
-    }
-    reduce(reducer)
-  }
-
-  /**
-   * Applies a CoWindow transformation on the connected DataStreams. The
-   * transformation calls the {@link CoWindowFunction#coWindow} method for for
-   * time aligned windows of the two data streams. System time is used as
-   * default to compute windows.
-   *
-   * @param coWindowFunction
-   * The { @link CoWindowFunction} that will be applied for the time
-   *             windows.
-   * @param windowSize
-   * Size of the windows that will be aligned for both streams in
-   * milliseconds.
-   * @param slideInterval
-   * After every function call the windows will be slid by this
-   * interval.
-   *
-   * @return The transformed { @link DataStream}.
-   */
-  def windowReduce[R: TypeInformation: ClassTag](coWindowFunction: 
-      CoWindowFunction[IN1, IN2, R], windowSize: Long, slideInterval: Long):
-      DataStream[R] = {
-    if (coWindowFunction == null) {
-      throw new NullPointerException("CoWindow function must no be null")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    
-    javaStream.windowReduce(coWindowFunction, windowSize, slideInterval).
-    returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a CoWindow transformation on the connected DataStreams. The
-   * transformation calls the {@link CoWindowFunction#coWindow} method for for
-   * time aligned windows of the two data streams. System time is used as
-   * default to compute windows.
-   *
-   * @param coWindower
-   * The coWindowing function to be applied for the time windows.
-   * @param windowSize
-   * Size of the windows that will be aligned for both streams in
-   * milliseconds.
-   * @param slideInterval
-   * After every function call the windows will be slid by this
-   * interval.
-   *
-   * @return The transformed { @link DataStream}.
-   */
-  def windowReduce[R: TypeInformation: ClassTag](coWindower: (Seq[IN1], Seq[IN2], 
-      Collector[R]) => Unit, windowSize: Long, slideInterval: Long):
-      DataStream[R] = {
-    if (coWindower == null) {
-      throw new NullPointerException("CoWindow function must no be null")
-    }
-
-    val cleanCoWindower = clean(coWindower)
-
-    val coWindowFun = new CoWindowFunction[IN1, IN2, R] {
-      def coWindow(first: util.List[IN1], second: util.List[IN2], 
-          out: Collector[R]): Unit = cleanCoWindower(first.asScala, second.asScala, out)
-    }
-
-    windowReduce(coWindowFun, windowSize, slideInterval)
-  }
-
-  /**
-   * Returns the first {@link DataStream}.
-   *
-   * @return The first DataStream.
-   */
-  def getFirst(): DataStream[IN1] = {
-    javaStream.getFirst
-  }
-
-  /**
-   * Returns the second {@link DataStream}.
-   *
-   * @return The second DataStream.
-   */
-  def getSecond(): DataStream[IN2] = {
-    javaStream.getSecond
-  }
-
-  /**
-   * Gets the type of the first input
-   *
-   * @return The type of the first input
-   */
-  def getInputType1(): TypeInformation[IN1] = {
-    javaStream.getType1
-  }
-
-  /**
-   * Gets the type of the second input
-   *
-   * @return The type of the second input
-   */
-  def getInputType2(): TypeInformation[IN2] = {
-    javaStream.getType2
-  }
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
new file mode 100644
index 0000000..41c1a7a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
@@ -0,0 +1,353 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream}
+import org.apache.flink.streaming.api.functions.co.{CoFlatMapFunction, CoMapFunction}
+import org.apache.flink.util.Collector
+import scala.reflect.ClassTag
+
+/**
+ * [[ConnectedStreams]] represents two connected streams of (possible) different data types. It
+ * can be used to apply transformations such as [[CoMapFunction]] on two
+ * [[DataStream]]s.
+ */
+class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
+
+  /**
+   * Applies a CoMap transformation on a {@link ConnectedStreams} and maps
+   * the output to a common type. The transformation calls a
+   * @param fun1 for each element of the first input and
+   * @param fun2 for each element of the second input. Each
+   * CoMapFunction call returns exactly one element.
+   *
+   * The CoMapFunction used to jointly transform the two input
+   * DataStreams
+   * @return The transformed { @link DataStream}
+   */
+  def map[R: TypeInformation: ClassTag](fun1: IN1 => R, fun2: IN2 => R): 
+  DataStream[R] = {
+    if (fun1 == null || fun2 == null) {
+      throw new NullPointerException("Map function must not be null.")
+    }
+    val cleanFun1 = clean(fun1)
+    val cleanFun2 = clean(fun2)
+    val comapper = new CoMapFunction[IN1, IN2, R] {
+      def map1(in1: IN1): R = cleanFun1(in1)
+      def map2(in2: IN2): R = cleanFun2(in2)
+    }
+
+    map(comapper)
+  }
+
+  /**
+   * Applies a CoMap transformation on a {@link ConnectedStreams} 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
+   * DataStreams
+   * @return The transformed { @link DataStream}
+   */
+  def map[R: TypeInformation: ClassTag](coMapper: CoMapFunction[IN1, IN2, R]): 
+  DataStream[R] = {
+    if (coMapper == null) {
+      throw new NullPointerException("Map function must not be null.")
+    }
+
+    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
+    javaStream.map(coMapper).returns(outType).asInstanceOf[JavaStream[R]]
+  }
+
+  /**
+   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
+   * maps the output to a common type. The transformation calls a
+   * {@link CoFlatMapFunction#flatMap1} for each element of the first input
+   * and {@link CoFlatMapFunction#flatMap2} 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
+   * DataStreams
+   * @return The transformed { @link DataStream}
+   */
+  def flatMap[R: TypeInformation: ClassTag](coFlatMapper: CoFlatMapFunction[IN1, IN2, R]): 
+  DataStream[R] = {
+    if (coFlatMapper == null) {
+      throw new NullPointerException("FlatMap function must not be null.")
+    }
+    
+    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
+    javaStream.flatMap(coFlatMapper).returns(outType).asInstanceOf[JavaStream[R]]
+  }
+
+  /**
+   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
+   * maps the output to a common type. The transformation calls a
+   * @param fun1 for each element of the first input
+   * and @param fun2 for each element of the second
+   * input. Each CoFlatMapFunction call returns any number of elements
+   * including none.
+   *
+   * @return The transformed { @link DataStream}
+   */
+  def flatMap[R: TypeInformation: ClassTag](fun1: (IN1, Collector[R]) => Unit, 
+      fun2: (IN2, Collector[R]) => Unit): DataStream[R] = {
+    if (fun1 == null || fun2 == null) {
+      throw new NullPointerException("FlatMap functions must not be null.")
+    }
+    val cleanFun1 = clean(fun1)
+    val cleanFun2 = clean(fun2)
+    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
+      def flatMap1(value: IN1, out: Collector[R]): Unit = cleanFun1(value, out)
+      def flatMap2(value: IN2, out: Collector[R]): Unit = cleanFun2(value, out)
+    }
+    flatMap(flatMapper)
+  }
+
+  /**
+   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
+   * maps the output to a common type. The transformation calls a
+   * @param fun1 for each element of the first input
+   * and @param fun2 for each element of the second
+   * input. Each CoFlatMapFunction call returns any number of elements
+   * including none.
+   *
+   * @return The transformed { @link DataStream}
+   */
+  def flatMap[R: TypeInformation: ClassTag](fun1: IN1 => TraversableOnce[R],
+      fun2: IN2 => TraversableOnce[R]): DataStream[R] = {
+    if (fun1 == null || fun2 == null) {
+      throw new NullPointerException("FlatMap functions must not be null.")
+    }
+    val cleanFun1 = clean(fun1)
+    val cleanFun2 = clean(fun2)
+    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
+      def flatMap1(value: IN1, out: Collector[R]) = { cleanFun1(value) foreach out.collect }
+      def flatMap2(value: IN2, out: Collector[R]) = { cleanFun2(value) foreach out.collect }
+    }
+    flatMap(flatMapper)
+  }
+
+  /**
+   * 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
+   * {@link ConnectedStreams#reduce}
+   *
+   * @param keyPosition1
+   * The field used to compute the hashcode of the elements in the
+   * first input stream.
+   * @param keyPosition2
+   * The field used to compute the hashcode of the elements in the
+   * second input stream.
+   * @return @return The transformed { @link ConnectedStreams}
+   */
+  def groupBy(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
+    javaStream.groupBy(keyPosition1, keyPosition2)
+  }
+
+  /**
+   * GroupBy operation for connected data stream. Groups the elements of
+   * input1 and input2 according to keyPositions1 and keyPositions2. Used for
+   * applying function on grouped data streams for example
+   * {@link ConnectedStreams#reduce}
+   *
+   * @param keyPositions1
+   * The fields used to group the first input stream.
+   * @param keyPositions2
+   * The fields used to group the second input stream.
+   * @return @return The transformed { @link ConnectedStreams}
+   */
+  def groupBy(keyPositions1: Array[Int], keyPositions2: Array[Int]): 
+  ConnectedStreams[IN1, IN2] = {
+    javaStream.groupBy(keyPositions1, keyPositions2)
+  }
+
+  /**
+   * GroupBy operation for connected data stream using key expressions. Groups
+   * the elements of input1 and input2 according to field1 and field2. A field
+   * expression is either the name of a public field or a getter method with
+   * parentheses of the {@link DataStream}S underlying type. A dot can be used
+   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+   *
+   * @param field1
+   * The grouping expression for the first input
+   * @param field2
+   * The grouping expression for the second input
+   * @return The grouped { @link ConnectedStreams}
+   */
+  def groupBy(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
+    javaStream.groupBy(field1, field2)
+  }
+
+  /**
+   * GroupBy operation for connected data stream using key expressions. Groups
+   * the elements of input1 and input2 according to fields1 and fields2. A
+   * field expression is either the name of a public field or a getter method
+   * with parentheses of the {@link DataStream}S underlying type. A dot can be
+   * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+   * .
+   *
+   * @param fields1
+   * The grouping expressions for the first input
+   * @param fields2
+   * The grouping expressions for the second input
+   * @return The grouped { @link ConnectedStreams}
+   */
+  def groupBy(fields1: Array[String], fields2: Array[String]): 
+  ConnectedStreams[IN1, IN2] = {
+    javaStream.groupBy(fields1, fields2)
+  }
+
+  /**
+   * GroupBy operation for connected data stream. Groups the elements of
+   * input1 and input2 using fun1 and fun2. Used for applying
+   * function on grouped data streams for example
+   * {@link ConnectedStreams#reduce}
+   *
+   * @param fun1
+   * The function used for grouping the first input
+   * @param fun2
+   * The function used for grouping the second input
+   * @return The grouped { @link ConnectedStreams}
+   */
+  def groupBy[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
+  ConnectedStreams[IN1, IN2] = {
+
+    val cleanFun1 = clean(fun1)
+    val cleanFun2 = clean(fun2)
+    val keyExtractor1 = new KeySelector[IN1, K] {
+      def getKey(in: IN1) = cleanFun1(in)
+    }
+    val keyExtractor2 = new KeySelector[IN2, L] {
+      def getKey(in: IN2) = cleanFun2(in)
+    }
+
+    javaStream.groupBy(keyExtractor1, keyExtractor2)
+  }
+
+  /**
+   * PartitionBy operation for connected data stream. Partitions the elements of
+   * input1 and input2 according to keyPosition1 and keyPosition2.
+   *
+   * @param keyPosition1
+   * The field used to compute the hashcode of the elements in the
+   * first input stream.
+   * @param keyPosition2
+   * The field used to compute the hashcode of the elements in the
+   * second input stream.
+   * @return The transformed { @link ConnectedStreams}
+   */
+  def partitionByHash(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
+    javaStream.partitionByHash(keyPosition1, keyPosition2)
+  }
+
+  /**
+   * PartitionBy operation for connected data stream. Partitions the elements of
+   * input1 and input2 according to keyPositions1 and keyPositions2.
+   *
+   * @param keyPositions1
+   * The fields used to partition the first input stream.
+   * @param keyPositions2
+   * The fields used to partition the second input stream.
+   * @return The transformed { @link ConnectedStreams}
+   */
+  def partitionByHash(keyPositions1: Array[Int], keyPositions2: Array[Int]):
+  ConnectedStreams[IN1, IN2] = {
+    javaStream.partitionByHash(keyPositions1, keyPositions2)
+  }
+
+  /**
+   * PartitionBy operation for connected data stream using key expressions. Partitions
+   * the elements of input1 and input2 according to field1 and field2. A field
+   * expression is either the name of a public field or a getter method with
+   * parentheses of the {@link DataStream}S underlying type. A dot can be used
+   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+   *
+   * @param field1
+   * The partitioning expression for the first input
+   * @param field2
+   * The partitioning expression for the second input
+   * @return The grouped { @link ConnectedStreams}
+   */
+  def partitionByHash(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
+    javaStream.partitionByHash(field1, field2)
+  }
+
+  /**
+   * PartitionBy operation for connected data stream using key expressions. Partitions
+   * the elements of input1 and input2 according to fields1 and fields2.
+   *
+   * @param fields1
+   * The partitioning expressions for the first input
+   * @param fields2
+   * The partitioning expressions for the second input
+   * @return The partitioned { @link ConnectedStreams}
+   */
+  def partitionByHash(fields1: Array[String], fields2: Array[String]):
+  ConnectedStreams[IN1, IN2] = {
+    javaStream.partitionByHash(fields1, fields2)
+  }
+
+  /**
+   * PartitionBy operation for connected data stream. Partitions the elements of
+   * input1 and input2 using fun1 and fun2.
+   *
+   * @param fun1
+   * The function used for partitioning the first input
+   * @param fun2
+   * The function used for partitioning the second input
+   * @return The partitioned { @link ConnectedStreams}
+   */
+  def partitionByHash[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
+  ConnectedStreams[IN1, IN2] = {
+
+    val cleanFun1 = clean(fun1)
+    val cleanFun2 = clean(fun2)
+
+    val keyExtractor1 = new KeySelector[IN1, K] {
+      def getKey(in: IN1) = cleanFun1(in)
+    }
+    val keyExtractor2 = new KeySelector[IN2, L] {
+      def getKey(in: IN2) = cleanFun2(in)
+    }
+
+    javaStream.partitionByHash(keyExtractor1, keyExtractor2)
+  }
+
+  /**
+   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+   * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
+   */
+  private[flink] def clean[F <: AnyRef](f: F): F = {
+    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index ca5fc48..c9aee61 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -203,11 +203,11 @@ class DataStream[T](javaStream: JavaStream[T]) {
     javaStream.union(dataStreams.map(_.getJavaStream): _*)
 
   /**
-   * Creates a new ConnectedDataStream by connecting
+   * Creates a new ConnectedStreams by connecting
    * DataStream outputs of different type with each other. The
    * DataStreams connected using this operators can be used with CoFunctions.
    */
-  def connect[T2](dataStream: DataStream[T2]): ConnectedDataStream[T, T2] = 
+  def connect[T2](dataStream: DataStream[T2]): ConnectedStreams[T, T2] =
     javaStream.connect(dataStream.getJavaStream)
 
 
@@ -408,7 +408,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * stream of the iterative part.
    * 
    * The input stream of the iterate operator and the feedback stream will be treated
-   * as a ConnectedDataStream where the the input is connected with the feedback stream.
+   * as a ConnectedStreams where the the input is connected with the feedback stream.
    * 
    * This allows the user to distinguish standard input from feedback inputs.
    * 
@@ -420,7 +420,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * to 0 then the iteration sources will indefinitely, so the job must be killed to stop.
    *
    */
-  def iterate[R, F: TypeInformation: ClassTag](stepFunction: ConnectedDataStream[T, F] => 
+  def iterate[R, F: TypeInformation: ClassTag](stepFunction: ConnectedStreams[T, F] =>
     (DataStream[F], DataStream[R]), maxWaitTimeMillis:Long): DataStream[R] = {
     val feedbackType: TypeInformation[F] = implicitly[TypeInformation[F]]
     val connectedIterativeStream = javaStream.iterate(maxWaitTimeMillis).
@@ -712,21 +712,6 @@ class DataStream[T](javaStream: JavaStream[T]) {
     new StreamJoinOperator[T, R](javaStream, stream.getJavaStream)
 
   /**
-   * Initiates a temporal cross transformation that builds all pair
-   * combinations of elements of both DataStreams, i.e., it builds a Cartesian
-   * product.
-   *
-   * This method returns a StreamJoinOperator on which the
-   * .onWindow(..) should be called to define the
-   * window, and then the .where(..) and .equalTo(..) methods can be used to defin
-   * the join keys.</p> The user can also use the apply method of the returned JoinedStream
-   * to use custom join function.
-   *
-   */
-  def cross[R](stream: DataStream[R]): StreamCrossOperator[T, R] =
-    new StreamCrossOperator[T, R](javaStream, stream.getJavaStream)
-
-  /**
    * Writes a DataStream to the standard output stream (stdout). For each
    * element of the DataStream the result of .toString is
    * written.

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
deleted file mode 100644
index 0060a9f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala
+++ /dev/null
@@ -1,101 +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.scala
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.api.common.functions.CrossFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.streaming.api.datastream.temporal.TemporalWindow
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
-import org.apache.flink.streaming.api.functions.co.CrossWindowFunction
-import org.apache.flink.streaming.api.operators.co.CoStreamWindow
-
-import scala.reflect.ClassTag
-
-class StreamCrossOperator[I1, I2](i1: JavaStream[I1], i2: JavaStream[I2]) extends
-  TemporalOperator[I1, I2, StreamCrossOperator.CrossWindow[I1, I2]](i1, i2) {
-
-  override def createNextWindowOperator(): StreamCrossOperator.CrossWindow[I1, I2] = {
-
-    val crossWindowFunction = StreamCrossOperator.getCrossWindowFunction(this,
-      (l: I1, r: I2) => (l, r))
-
-
-    val returnType = createTuple2TypeInformation[I1, I2](input1.getType, input2.getType)
-    val javaStream = input1.connect(input2).addGeneralWindowCombine(
-      crossWindowFunction,
-      returnType, windowSize,
-      slideInterval, timeStamp1, timeStamp2)
-
-    new StreamCrossOperator.CrossWindow[I1, I2](this, javaStream)
-  }
-}
-object StreamCrossOperator {
-
-  private[flink] class CrossWindow[I1, I2](op: StreamCrossOperator[I1, I2],
-                                           javaStream: JavaStream[(I1, I2)]) extends
-    DataStream[(I1, I2)](javaStream) with TemporalWindow[CrossWindow[I1, I2]] {
-
-    /**
-     * Sets a wrapper for the crossed elements. For each crossed pair, the result of the udf
-     * call will be emitted.
-     *
-     */
-    def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = {
-
-      val cleanCrossWindowFunction = clean(getCrossWindowFunction(op, fun))
-
-      op.input1.connect(op.input2).addGeneralWindowCombine(
-        cleanCrossWindowFunction,
-        implicitly[TypeInformation[R]],
-        op.windowSize,
-        op.slideInterval,
-        op.timeStamp1,
-        op.timeStamp2)
-    }
-    
-    override def every(length: Long, timeUnit: TimeUnit): CrossWindow[I1, I2] = {
-      every(timeUnit.toMillis(length))
-    }
-
-    override def every(length: Long): CrossWindow[I1, I2] = {
-      val graph = javaStream.getExecutionEnvironment().getStreamGraph()
-      val operator = graph.getStreamNode(javaStream.getId()).getOperator()
-      operator.asInstanceOf[CoStreamWindow[_,_,_]].setSlideSize(length)
-      this
-    }
-  }
-
-  private[flink] def getCrossWindowFunction[I1, I2, R](op: StreamCrossOperator[I1, I2],
-                                                       crossFunction: (I1, I2) => R):
-  CrossWindowFunction[I1, I2, R] = {
-    require(crossFunction != null, "Join function must not be null.")
-
-    val cleanFun = op.input1.clean(crossFunction)
-    val crossFun = new CrossFunction[I1, I2, R] {
-      override def cross(first: I1, second: I2): R = {
-        cleanFun(first, second)
-      }
-    }
-
-    new CrossWindowFunction[I1, I2, R](crossFun)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
index e2be44f..f584767 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
@@ -26,7 +26,6 @@ import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.java.operators.Keys
 import org.apache.flink.streaming.api.datastream.temporal.TemporalWindow
 import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
-import org.apache.flink.streaming.api.functions.co.JoinWindowFunction
 import org.apache.flink.streaming.util.keys.KeySelectorUtil
 
 import scala.Array.canBuildFrom
@@ -151,10 +150,11 @@ object StreamJoinOperator {
 
     private def createJoinOperator(): JavaStream[(I1, I2)] = {
 
-      val returnType = createTuple2TypeInformation[I1, I2](op.input1.getType, op.input2.getType)
-      op.input1.groupBy(keys1).connect(op.input2.groupBy(keys2))
-        .addGeneralWindowCombine(getJoinWindowFunction(this, (_, _)),
-          returnType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)
+//      val returnType = createTuple2TypeInformation[I1, I2](op.input1.getType, op.input2.getType)
+//      op.input1.groupBy(keys1).connect(op.input2.groupBy(keys2))
+//        .addGeneralWindowCombine(getJoinWindowFunction(this, (_, _)),
+//          returnType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)
+      null
     }
   }
 
@@ -172,14 +172,15 @@ object StreamJoinOperator {
 
       val cleanFun = clean(getJoinWindowFunction(jp, fun))
 
-      op.input1.groupBy(jp.keys1).connect(op.input2.groupBy(jp.keys2))
-        .addGeneralWindowCombine[R](
-          cleanFun,
-          implicitly[TypeInformation[R]],
-          op.windowSize,
-          op.slideInterval,
-          op.timeStamp1,
-          op.timeStamp2)
+//      op.input1.groupBy(jp.keys1).connect(op.input2.groupBy(jp.keys2))
+//        .addGeneralWindowCombine[R](
+//          cleanFun,
+//          implicitly[TypeInformation[R]],
+//          op.windowSize,
+//          op.slideInterval,
+//          op.timeStamp1,
+//          op.timeStamp2)
+      null
     }
   }
 
@@ -195,7 +196,8 @@ object StreamJoinOperator {
       }
     }
 
-    new JoinWindowFunction[I1, I2, R](jp.keys1, jp.keys2, joinFun)
+//    new JoinWindowFunction[I1, I2, R](jp.keys1, jp.keys2, joinFun)
+    null
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
index 59843e2..b8a3b94 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
@@ -26,7 +26,7 @@ import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
 import org.apache.flink.streaming.api.datastream.{ WindowedDataStream => JavaWStream }
 import org.apache.flink.streaming.api.datastream.{ SplitDataStream => SplitJavaStream }
-import org.apache.flink.streaming.api.datastream.{ ConnectedDataStream => JavaConStream }
+import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => JavaConStream }
 import org.apache.flink.streaming.api.datastream.{ GroupedDataStream => GroupedJavaStream }
 import language.implicitConversions
 
@@ -48,7 +48,7 @@ package object scala {
     new SplitDataStream[R](javaStream)
 
   implicit def javaToScalaConnectedStream[IN1, IN2](javaStream: JavaConStream[IN1, IN2]):
-  ConnectedDataStream[IN1, IN2] = new ConnectedDataStream[IN1, IN2](javaStream)
+  ConnectedStreams[IN1, IN2] = new ConnectedStreams[IN1, IN2](javaStream)
 
   implicit def seqToFlinkSource[T: ClassTag: TypeInformation](scalaSeq: Seq[T]) : DataStream[T] =
     StreamExecutionEnvironment.getExecutionEnvironment.fromCollection(scalaSeq)

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index 8b4d527..606aac5 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -84,7 +84,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
   /**
    * Tests that {@link DataStream#groupBy} and {@link DataStream#partitionBy(KeySelector)} result in
-   * different and correct topologies. Does the some for the {@link ConnectedDataStream}.
+   * different and correct topologies. Does the some for the {@link ConnectedStreams}.
    */
   @Test
   def testPartitioning(): Unit = {
@@ -144,21 +144,21 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid2)))
     assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid3)))
 
-    //Testing ConnectedDataStream grouping
-    val connectedGroup1: ConnectedDataStream[_, _] = connected.groupBy(0, 0)
+    //Testing ConnectedStreams grouping
+    val connectedGroup1: ConnectedStreams[_, _] = connected.groupBy(0, 0)
     val downStreamId1: Integer = createDownStreamId(connectedGroup1)
 
-    val connectedGroup2: ConnectedDataStream[_, _] = connected.groupBy(Array[Int](0), Array[Int](0))
+    val connectedGroup2: ConnectedStreams[_, _] = connected.groupBy(Array[Int](0), Array[Int](0))
     val downStreamId2: Integer = createDownStreamId(connectedGroup2)
 
-    val connectedGroup3: ConnectedDataStream[_, _] = connected.groupBy("_1", "_1")
+    val connectedGroup3: ConnectedStreams[_, _] = connected.groupBy("_1", "_1")
     val downStreamId3: Integer = createDownStreamId(connectedGroup3)
 
-    val connectedGroup4: ConnectedDataStream[_, _] =
+    val connectedGroup4: ConnectedStreams[_, _] =
       connected.groupBy(Array[String]("_1"), Array[String]("_1"))
     val downStreamId4: Integer = createDownStreamId(connectedGroup4)
 
-    val connectedGroup5: ConnectedDataStream[_, _] = connected.groupBy(x => x._1, x => x._1)
+    val connectedGroup5: ConnectedStreams[_, _] = connected.groupBy(x => x._1, x => x._1)
     val downStreamId5: Integer = createDownStreamId(connectedGroup5)
 
     assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId1)))
@@ -176,22 +176,22 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId5)))
     assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId5)))
 
-    //Testing ConnectedDataStream partitioning
-    val connectedPartition1: ConnectedDataStream[_, _] = connected.partitionByHash(0, 0)
+    //Testing ConnectedStreams partitioning
+    val connectedPartition1: ConnectedStreams[_, _] = connected.partitionByHash(0, 0)
     val connectDownStreamId1: Integer = createDownStreamId(connectedPartition1)
 
-    val connectedPartition2: ConnectedDataStream[_, _] =
+    val connectedPartition2: ConnectedStreams[_, _] =
       connected.partitionByHash(Array[Int](0), Array[Int](0))
     val connectDownStreamId2: Integer = createDownStreamId(connectedPartition2)
 
-    val connectedPartition3: ConnectedDataStream[_, _] = connected.partitionByHash("_1", "_1")
+    val connectedPartition3: ConnectedStreams[_, _] = connected.partitionByHash("_1", "_1")
     val connectDownStreamId3: Integer = createDownStreamId(connectedPartition3)
 
-    val connectedPartition4: ConnectedDataStream[_, _] =
+    val connectedPartition4: ConnectedStreams[_, _] =
       connected.partitionByHash(Array[String]("_1"), Array[String]("_1"))
     val connectDownStreamId4: Integer = createDownStreamId(connectedPartition4)
 
-    val connectedPartition5: ConnectedDataStream[_, _] =
+    val connectedPartition5: ConnectedStreams[_, _] =
       connected.partitionByHash(x => x._1, x => x._1)
     val connectDownStreamId5: Integer = createDownStreamId(connectedPartition5)
 
@@ -492,7 +492,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     // we need to rebalance before iteration
     val source = env.fromElements(1, 2, 3).map { t: Int => t }
 
-    val iterated = source.iterate((input: ConnectedDataStream[Int, String]) => {
+    val iterated = source.iterate((input: ConnectedStreams[Int, String]) => {
       val head = input.map(i => (i + 1).toString, s => s)
       (head.filter(_ == "2"), head.filter(_ != "2"))
     }, 1000).print()
@@ -501,7 +501,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
       (input.map(_ + 1), input.map(_.toString)), 2000)
 
     try {
-      val invalid = source.iterate((input: ConnectedDataStream[Int, String]) => {
+      val invalid = source.iterate((input: ConnectedStreams[Int, String]) => {
         val head = input.partitionByHash(1, 1).map(i => (i + 1).toString, s => s)
         (head.filter(_ == "2"), head.filter(_ != "2"))
       }, 1000).print()
@@ -546,7 +546,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     return dataStream.print.getTransformation.getId
   }
 
-  private def createDownStreamId(dataStream: ConnectedDataStream[_, _]): Integer = {
+  private def createDownStreamId(dataStream: ConnectedStreams[_, _]): Integer = {
     val m = dataStream.map(x => 0, x => 0)
     m.print()
     m.getId

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index f53b986..66fe197 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -43,11 +43,11 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "org.apache.flink.streaming.api.datastream.DataStream.transform",
       "org.apache.flink.streaming.api.datastream.DataStream.getTransformation",
       "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.copy",
-      "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getType1",
-      "org.apache.flink.streaming.api.datastream.ConnectedDataStream.getType2",
-      "org.apache.flink.streaming.api.datastream.ConnectedDataStream.addGeneralWindowCombine",
-      "org.apache.flink.streaming.api.datastream.ConnectedDataStream.transform",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType1",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType2",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.addGeneralWindowCombine",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.transform",
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getType",
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig",
 
@@ -104,9 +104,9 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       classOf[DataStream[_]])
 
     checkMethods(
-      "ConnectedDataStream", "ConnectedDataStream",
-      classOf[org.apache.flink.streaming.api.datastream.ConnectedDataStream[_,_]],
-      classOf[ConnectedDataStream[_,_]])
+      "ConnectedStreams", "ConnectedStreams",
+      classOf[org.apache.flink.streaming.api.datastream.ConnectedStreams[_,_]],
+      classOf[ConnectedStreams[_,_]])
 
     checkMethods(
       "SplitDataStream", "SplitDataStream",
@@ -114,11 +114,6 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       classOf[SplitDataStream[_]])
 
     checkMethods(
-      "StreamCrossOperator", "StreamCrossOperator",
-      classOf[org.apache.flink.streaming.api.datastream.temporal.StreamCrossOperator[_,_]],
-      classOf[StreamCrossOperator[_,_]])
-
-    checkMethods(
       "StreamJoinOperator", "StreamJoinOperator",
       classOf[org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator[_,_]],
       classOf[StreamJoinOperator[_,_]])


[11/13] flink git commit: [FLINK-2550] Change Window API constructs to use Time instead of long

Posted by al...@apache.org.
[FLINK-2550] Change Window API constructs to use Time instead of long

This covers assigners/triggers/evictors.


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

Branch: refs/heads/master
Commit: 8c2c76947a9a26e25d4539068b5253b265c71c23
Parents: 833b347
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Oct 3 16:39:13 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    | 10 +++---
 .../streaming/api/datastream/KeyedStream.java   |  8 ++---
 .../assigners/SlidingProcessingTimeWindows.java |  5 +--
 .../windowing/assigners/SlidingTimeWindows.java |  5 +--
 .../TumblingProcessingTimeWindows.java          |  5 +--
 .../assigners/TumblingTimeWindows.java          |  5 +--
 .../api/windowing/evictors/TimeEvictor.java     |  5 +--
 .../api/windowing/time/AbstractTime.java        |  3 ++
 .../ContinuousProcessingTimeTrigger.java        | 31 +++++++++---------
 .../triggers/ContinuousWatermarkTrigger.java    | 25 ++++++++-------
 .../windowing/AllWindowTranslationTest.java     | 17 +++++-----
 .../windowing/NonKeyedWindowOperatorTest.java   | 25 ++++++---------
 .../operators/windowing/WindowOperatorTest.java | 16 +++++-----
 .../windowing/WindowTranslationTest.java        | 18 ++++++-----
 .../flink/streaming/api/scala/DataStream.scala  | 12 +++----
 .../flink/streaming/api/scala/KeyedStream.scala | 12 +++----
 .../api/scala/AllWindowTranslationTest.scala    | 33 +++++++++++++-------
 .../api/scala/WindowTranslationTest.scala       | 24 +++++++++-----
 18 files changed, 144 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 6d88416..32d9012 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -741,9 +741,9 @@ public class DataStream<T> {
 		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
 		if (actualSize instanceof EventTime) {
-			return windowAll(TumblingTimeWindows.of(actualSize.toMilliseconds()));
+			return windowAll(TumblingTimeWindows.of(actualSize));
 		} else {
-			return windowAll(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
+			return windowAll(TumblingProcessingTimeWindows.of(actualSize));
 		}
 	}
 
@@ -763,11 +763,9 @@ public class DataStream<T> {
 		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
 		if (actualSize instanceof EventTime) {
-			return windowAll(SlidingTimeWindows.of(actualSize.toMilliseconds(),
-					actualSlide.toMilliseconds()));
+			return windowAll(SlidingTimeWindows.of(size, slide));
 		} else {
-			return windowAll(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(),
-					actualSlide.toMilliseconds()));
+			return windowAll(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
index 265886b..f7c5b53 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -125,9 +125,9 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
 		if (actualSize instanceof EventTime) {
-			return window(TumblingTimeWindows.of(actualSize.toMilliseconds()));
+			return window(TumblingTimeWindows.of(actualSize));
 		} else {
-			return window(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
+			return window(TumblingProcessingTimeWindows.of(actualSize));
 		}
 	}
 
@@ -147,9 +147,9 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
 		if (actualSize instanceof EventTime) {
-			return window(SlidingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+			return window(SlidingTimeWindows.of(actualSize, actualSlide));
 		} else {
-			return window(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+			return window(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
index a2d95c2..6fc79b0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.api.windowing.assigners;
 
 import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -86,7 +87,7 @@ public class SlidingProcessingTimeWindows extends WindowAssigner<Object, TimeWin
 	 * @param slide The slide interval of the generated windows.
 	 * @return The time policy.
 	 */
-	public static SlidingProcessingTimeWindows of(long size, long slide) {
-		return new SlidingProcessingTimeWindows(size, slide);
+	public static SlidingProcessingTimeWindows of(AbstractTime size, AbstractTime slide) {
+		return new SlidingProcessingTimeWindows(size.toMilliseconds(), slide.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
index cb5a7a1..49bff05 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.triggers.WatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -75,7 +76,7 @@ public class SlidingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	 * @param slide The slide interval of the generated windows.
 	 * @return The time policy.
 	 */
-	public static SlidingTimeWindows of(long size, long slide) {
-		return new SlidingTimeWindows(size, slide);
+	public static SlidingTimeWindows of(AbstractTime size, AbstractTime slide) {
+		return new SlidingTimeWindows(size.toMilliseconds(), slide.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
index b1ef857..1f2eebf 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -61,7 +62,7 @@ public class TumblingProcessingTimeWindows extends WindowAssigner<Object, TimeWi
 	 * @param size The size of the generated windows.
 	 * @return The time policy.
 	 */
-	public static TumblingProcessingTimeWindows of(long size) {
-		return new TumblingProcessingTimeWindows(size);
+	public static TumblingProcessingTimeWindows of(AbstractTime size) {
+		return new TumblingProcessingTimeWindows(size.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
index d19c97d..019f45b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.triggers.WatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -60,8 +61,8 @@ public class TumblingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	 * @param size The size of the generated windows.
 	 * @return The time policy.
 	 */
-	public static TumblingTimeWindows of(long size) {
-		return new TumblingTimeWindows(size);
+	public static TumblingTimeWindows of(AbstractTime size) {
+		return new TumblingTimeWindows(size.toMilliseconds());
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
index c38100c..2965214 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.api.windowing.evictors;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
@@ -54,7 +55,7 @@ public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
 		return windowSize;
 	}
 
-	public static <W extends Window> TimeEvictor<W> of(long windowSize) {
-		return new TimeEvictor<>(windowSize);
+	public static <W extends Window> TimeEvictor<W> of(AbstractTime windowSize) {
+		return new TimeEvictor<>(windowSize.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
index 1264c2a..3f8fb60 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
@@ -24,6 +24,9 @@ import java.util.concurrent.TimeUnit;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
+/**
+ * Base class for {@link Time} implementations.
+ */
 public abstract class AbstractTime {
 
 	/** The time unit for this policy's time interval */

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
index da198be..24e8ce3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
@@ -18,32 +18,33 @@
 package org.apache.flink.streaming.api.windowing.triggers;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
 public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigger<Object, W> {
 	private static final long serialVersionUID = 1L;
 
-	private long granularity;
+	private long interval;
 
 	private long nextFireTimestamp = 0;
 
-	private ContinuousProcessingTimeTrigger(long granularity) {
-		this.granularity = granularity;
+	private ContinuousProcessingTimeTrigger(long interval) {
+		this.interval = interval;
 	}
 
 	@Override
 	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) {
 		long currentTime = System.currentTimeMillis();
 		if (nextFireTimestamp == 0) {
-			long start = currentTime - (currentTime % granularity);
-			nextFireTimestamp = start + granularity;
+			long start = currentTime - (currentTime % interval);
+			nextFireTimestamp = start + interval;
 
 			ctx.registerProcessingTimeTimer(nextFireTimestamp);
 			return TriggerResult.CONTINUE;
 		}
 		if (currentTime > nextFireTimestamp) {
-			long start = currentTime - (currentTime % granularity);
-			nextFireTimestamp = start + granularity;
+			long start = currentTime - (currentTime % interval);
+			nextFireTimestamp = start + interval;
 
 			ctx.registerProcessingTimeTimer(nextFireTimestamp);
 
@@ -57,8 +58,8 @@ public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigge
 		// only fire if an element didn't already fire
 		long currentTime = System.currentTimeMillis();
 		if (currentTime > nextFireTimestamp) {
-			long start = currentTime - (currentTime % granularity);
-			nextFireTimestamp = start + granularity;
+			long start = currentTime - (currentTime % interval);
+			nextFireTimestamp = start + interval;
 			return TriggerResult.FIRE;
 		}
 		return TriggerResult.CONTINUE;
@@ -66,20 +67,20 @@ public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigge
 
 	@Override
 	public Trigger<Object, W> duplicate() {
-		return new ContinuousProcessingTimeTrigger<>(granularity);
+		return new ContinuousProcessingTimeTrigger<>(interval);
 	}
 
 	@VisibleForTesting
-	public long getGranularity() {
-		return granularity;
+	public long getInterval() {
+		return interval;
 	}
 
 	@Override
 	public String toString() {
-		return "ContinuousProcessingTimeTrigger(" + granularity + ")";
+		return "ContinuousProcessingTimeTrigger(" + interval + ")";
 	}
 
-	public static <W extends Window> ContinuousProcessingTimeTrigger<W> of(long granularity) {
-		return new ContinuousProcessingTimeTrigger<>(granularity);
+	public static <W extends Window> ContinuousProcessingTimeTrigger<W> of(AbstractTime interval) {
+		return new ContinuousProcessingTimeTrigger<>(interval.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
index 3b6dc6d..e11ceba 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
@@ -18,24 +18,25 @@
 package org.apache.flink.streaming.api.windowing.triggers;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
 public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Object, W> {
 	private static final long serialVersionUID = 1L;
 
-	private long granularity;
+	private long interval;
 
 	private boolean first = true;
 
-	private ContinuousWatermarkTrigger(long granularity) {
-		this.granularity = granularity;
+	private ContinuousWatermarkTrigger(long interval) {
+		this.interval = interval;
 	}
 
 	@Override
 	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) {
 		if (first) {
-			long start = timestamp - (timestamp % granularity);
-			long nextFireTimestamp = start + granularity;
+			long start = timestamp - (timestamp % interval);
+			long nextFireTimestamp = start + interval;
 
 			ctx.registerWatermarkTimer(nextFireTimestamp);
 			first = false;
@@ -46,26 +47,26 @@ public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Obj
 
 	@Override
 	public TriggerResult onTime(long time, TriggerContext ctx) {
-		ctx.registerWatermarkTimer(time + granularity);
+		ctx.registerWatermarkTimer(time + interval);
 		return TriggerResult.FIRE;
 	}
 
 	@Override
 	public Trigger<Object, W> duplicate() {
-		return new ContinuousWatermarkTrigger<>(granularity);
+		return new ContinuousWatermarkTrigger<>(interval);
 	}
 
 	@Override
 	public String toString() {
-		return "ContinuousProcessingTimeTrigger(" + granularity + ")";
+		return "ContinuousProcessingTimeTrigger(" + interval + ")";
 	}
 
 	@VisibleForTesting
-	public long getGranularity() {
-		return granularity;
+	public long getInterval() {
+		return interval;
 	}
 
-	public static <W extends Window> ContinuousWatermarkTrigger<W> of(long granularity) {
-		return new ContinuousWatermarkTrigger<>(granularity);
+	public static <W extends Window> ContinuousWatermarkTrigger<W> of(AbstractTime interval) {
+		return new ContinuousWatermarkTrigger<>(interval.toMilliseconds());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
index ee8c6d6..767b40c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeW
 import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
 import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -39,6 +40,8 @@ import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * These tests verify that the api calls on
  * {@link org.apache.flink.streaming.api.datastream.AllWindowedStream} instantiate
@@ -62,7 +65,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DummyReducer reducer = new DummyReducer();
 
 		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.reduceWindow(reducer);
 
 		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
@@ -70,7 +73,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
@@ -98,7 +101,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DummyReducer reducer = new DummyReducer();
 
 		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
 				.reduceWindow(reducer);
 
@@ -111,7 +114,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
 				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
@@ -144,7 +147,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DummyReducer reducer = new DummyReducer();
 
 		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
 				.reduceWindow(reducer);
 
@@ -158,9 +161,9 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(100))
+				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
index 9b0bcc4..6cc8931 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.triggers.ContinuousWatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
@@ -49,6 +50,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 @RunWith(Parameterized.class)
@@ -69,11 +71,11 @@ public class NonKeyedWindowOperatorTest {
 	public void testSlidingEventTimeWindows() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
-		final int WINDOW_SLIDE = 1000;
+		final int WINDOW_SIZE = 3;
+		final int WINDOW_SLIDE = 1;
 
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
-				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
+				SlidingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
 				windowBufferFactory,
 				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
@@ -150,10 +152,10 @@ public class NonKeyedWindowOperatorTest {
 	public void testTumblingEventTimeWindows() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
+		final int WINDOW_SIZE = 3;
 
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
-				TumblingTimeWindows.of(WINDOW_SIZE),
+				TumblingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
 				windowBufferFactory,
 				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
@@ -228,13 +230,13 @@ public class NonKeyedWindowOperatorTest {
 	public void testContinuousWatermarkTrigger() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
+		final int WINDOW_SIZE = 3;
 
 		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
 				GlobalWindows.create(),
 				windowBufferFactory,
 				new ReduceAllWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
-				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
+				ContinuousWatermarkTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
 
@@ -421,13 +423,4 @@ public class NonKeyedWindowOperatorTest {
 			}
 		}
 	}
-
-	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple2<String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index 1bfd1d5..d387df0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
@@ -49,6 +50,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 @RunWith(Parameterized.class)
@@ -69,11 +71,11 @@ public class WindowOperatorTest {
 	public void testSlidingEventTimeWindows() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
-		final int WINDOW_SLIDE = 1000;
+		final int WINDOW_SIZE = 3;
+		final int WINDOW_SLIDE = 1;
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
+				SlidingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
 				new TupleKeySelector(),
 				windowBufferFactory,
 				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
@@ -157,10 +159,10 @@ public class WindowOperatorTest {
 	public void testTumblingEventTimeWindows() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
+		final int WINDOW_SIZE = 3;
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingTimeWindows.of(WINDOW_SIZE),
+				TumblingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
 				new TupleKeySelector(),
 				windowBufferFactory,
 				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
@@ -240,14 +242,14 @@ public class WindowOperatorTest {
 	public void testContinuousWatermarkTrigger() throws Exception {
 		closeCalled.set(0);
 
-		final int WINDOW_SIZE = 3000;
+		final int WINDOW_SIZE = 3;
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new WindowOperator<>(
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				windowBufferFactory,
 				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
-				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
+				ContinuousWatermarkTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index a3e6085..9dc6687 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeW
 import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
 import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -40,6 +41,8 @@ import org.apache.flink.util.Collector;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * These tests verify that the api calls on
  * {@link WindowedStream} instantiate
@@ -61,7 +64,8 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS),
+						Time.of(100, TimeUnit.MILLISECONDS)))
 				.reduceWindow(reducer);
 
 		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
@@ -70,7 +74,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
@@ -99,7 +103,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
 				.reduceWindow(reducer);
 
@@ -113,7 +117,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(1000))
+				.window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
 				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
@@ -147,7 +151,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
 				.reduceWindow(reducer);
 
@@ -162,9 +166,9 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(1000))
+				.window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(100))
+				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 7dfaeef..6ad7629 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -629,11 +629,11 @@ class DataStream[T](javaStream: JavaStream[T]) {
 
     actualSize match {
       case t: EventTime =>
-        val assigner = TumblingTimeWindows.of(actualSize.toMilliseconds)
+        val assigner = TumblingTimeWindows.of(actualSize)
           .asInstanceOf[WindowAssigner[T, TimeWindow]]
         windowAll(assigner)
       case t: ProcessingTime =>
-        val assigner = TumblingProcessingTimeWindows.of(actualSize.toMilliseconds)
+        val assigner = TumblingProcessingTimeWindows.of(actualSize)
           .asInstanceOf[WindowAssigner[T, TimeWindow]]
         windowAll(assigner)
       case _ => throw new RuntimeException("Invalid time: " + actualSize)
@@ -658,13 +658,13 @@ class DataStream[T](javaStream: JavaStream[T]) {
     actualSize match {
       case t: EventTime =>
         val assigner = SlidingTimeWindows.of(
-          actualSize.toMilliseconds,
-          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+          actualSize,
+          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
         windowAll(assigner)
       case t: ProcessingTime =>
         val assigner = SlidingProcessingTimeWindows.of(
-          actualSize.toMilliseconds,
-          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+          actualSize,
+          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
         windowAll(assigner)
       case _ => throw new RuntimeException("Invalid time: " + actualSize)
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index 232e4bb..18b71be 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -55,11 +55,11 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
 
     actualSize match {
       case t: EventTime =>
-        val assigner = TumblingTimeWindows.of(actualSize.toMilliseconds)
+        val assigner = TumblingTimeWindows.of(actualSize)
           .asInstanceOf[WindowAssigner[T, TimeWindow]]
         window(assigner)
       case t: ProcessingTime =>
-        val assigner = TumblingProcessingTimeWindows.of(actualSize.toMilliseconds)
+        val assigner = TumblingProcessingTimeWindows.of(actualSize)
           .asInstanceOf[WindowAssigner[T, TimeWindow]]
         window(assigner)
       case _ => throw new RuntimeException("Invalid time: " + actualSize)
@@ -85,13 +85,13 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
     actualSize match {
       case t: EventTime =>
         val assigner = SlidingTimeWindows.of(
-          actualSize.toMilliseconds,
-          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+          actualSize,
+          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
         window(assigner)
       case t: ProcessingTime =>
         val assigner = SlidingProcessingTimeWindows.of(
-          actualSize.toMilliseconds,
-          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+          actualSize,
+          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
         window(assigner)
       case _ => throw new RuntimeException("Invalid time: " + actualSize)
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
index 35c7fcc..247256f 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
@@ -19,13 +19,15 @@
 package org.apache.flink.streaming.api.scala
 
 
+import java.util.concurrent.TimeUnit
+
 import org.apache.flink.api.common.functions.RichReduceFunction
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction}
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
 import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
 import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
-import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, WatermarkTrigger, CountTrigger}
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
 import org.apache.flink.streaming.runtime.operators.windowing._
@@ -53,7 +55,9 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+      .windowAll(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
       .reduceWindow(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
@@ -65,10 +69,11 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(1000, 100))
-      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
+      .windowAll(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
+      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
       def apply(
-                    tuple: Tuple,
                     window: TimeWindow,
                     values: java.lang.Iterable[(String, Int)],
                     out: Collector[(String, Int)]) { }
@@ -91,7 +96,9 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+      .windowAll(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
       .reduceWindow(reducer)
 
@@ -109,7 +116,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 
     val window2 = source
-      .windowAll(TumblingProcessingTimeWindows.of(1000))
+      .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
       def apply(
@@ -139,8 +146,10 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
-      .evictor(TimeEvictor.of(1000))
+      .windowAll(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
+      .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
       .reduceWindow(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
@@ -157,7 +166,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 
     val window2 = source
-      .windowAll(TumblingProcessingTimeWindows.of(1000))
+      .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .evictor(CountEvictor.of(1000))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {

http://git-wip-us.apache.org/repos/asf/flink/blob/8c2c7694/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
index 49d0a1a..f1b05c6 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
@@ -18,12 +18,14 @@
 
 package org.apache.flink.streaming.api.scala
 
+import java.util.concurrent.TimeUnit
 
 import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
 import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
 import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
+import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
@@ -50,7 +52,9 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .window(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
       .reduceWindow(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
@@ -62,7 +66,9 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .window(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
         def apply(
             key: Tuple,
@@ -89,7 +95,9 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .window(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
       .reduceWindow(reducer)
 
@@ -108,7 +116,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .window(TumblingProcessingTimeWindows.of(1000))
+      .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
       def apply(
@@ -140,8 +148,10 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(1000, 100))
-      .evictor(TimeEvictor.of(1000))
+      .window(SlidingProcessingTimeWindows.of(
+        Time.of(1, TimeUnit.SECONDS),
+        Time.of(100, TimeUnit.MILLISECONDS)))
+      .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
       .reduceWindow(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
@@ -159,7 +169,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .window(TumblingProcessingTimeWindows.of(1000))
+      .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .evictor(CountEvictor.of(1000))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {


[02/13] flink git commit: [hotfix] Use closure cleaner for reduce window

Posted by al...@apache.org.
[hotfix] Use closure cleaner for reduce window


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

Branch: refs/heads/master
Commit: 0de9d2ef66c9f111633c1a95dfc8e3100098b4df
Parents: 62df0a0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 5 10:21:15 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/datastream/AllWindowedStream.java      | 6 ++++++
 .../apache/flink/streaming/api/datastream/WindowedStream.java  | 3 +++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0de9d2ef/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index 134029f..78ba8ad 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -121,6 +121,9 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @return The data stream that is the result of applying the reduce function to the window. 
 	 */
 	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "Reduce at " + callLocation;
 
@@ -185,6 +188,9 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
 	public <R> DataStream<R> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "MapWindow at " + callLocation;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0de9d2ef/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 41adab5..349651e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -132,6 +132,9 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @return The data stream that is the result of applying the reduce function to the window. 
 	 */
 	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "Reduce at " + callLocation;
 


[12/13] flink git commit: [FLINK-2550] Rename IterativeDataStream to IterativeStream

Posted by al...@apache.org.
[FLINK-2550] Rename IterativeDataStream to IterativeStream


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

Branch: refs/heads/master
Commit: 7b6e762fda09e5edb1d5cca2398cb8035d6794b4
Parents: 0de9d2e
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 5 11:42:53 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |  28 +--
 .../api/datastream/IterativeDataStream.java     | 218 -------------------
 .../api/datastream/IterativeStream.java         | 218 +++++++++++++++++++
 .../apache/flink/streaming/api/IterateTest.java |  38 ++--
 .../api/complex/ComplexIntegrationTest.java     |   4 +-
 .../examples/iteration/IterateExample.java      |   4 +-
 6 files changed, 255 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 32d9012..003ef36 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -467,14 +467,14 @@ public class DataStream<T> {
 	/**
 	 * Initiates an iterative part of the program that feeds back data streams.
 	 * The iterative part needs to be closed by calling
-	 * {@link IterativeDataStream#closeWith(DataStream)}. The transformation of
-	 * this IterativeDataStream will be the iteration head. The data stream
-	 * given to the {@link IterativeDataStream#closeWith(DataStream)} method is
+	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
+	 * this IterativeStream will be the iteration head. The data stream
+	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
 	 * the data stream that will be fed back and used as the input for the
 	 * iteration head. The user can also use different feedback type than the
 	 * input of the iteration and treat the input and feedback streams as a
 	 * {@link ConnectedStreams} be calling
-	 * {@link IterativeDataStream#withFeedbackType(TypeInformation)}
+	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
 	 * <p>
 	 * A common usage pattern for streaming iterations is to use output
 	 * splitting to send a part of the closing data stream to the head. Refer to
@@ -482,7 +482,7 @@ public class DataStream<T> {
 	 * <p>
 	 * The iteration edge will be partitioned the same way as the first input of
 	 * the iteration head unless it is changed in the
-	 * {@link IterativeDataStream#closeWith(DataStream)} call.
+	 * {@link IterativeStream#closeWith(DataStream)} call.
 	 * <p>
 	 * By default a DataStream with iteration will never terminate, but the user
 	 * can use the maxWaitTime parameter to set a max waiting time for the
@@ -491,21 +491,21 @@ public class DataStream<T> {
 	 * 
 	 * @return The iterative data stream created.
 	 */
-	public IterativeDataStream<T> iterate() {
-		return new IterativeDataStream<T>(this, 0);
+	public IterativeStream<T> iterate() {
+		return new IterativeStream<T>(this, 0);
 	}
 
 	/**
 	 * Initiates an iterative part of the program that feeds back data streams.
 	 * The iterative part needs to be closed by calling
-	 * {@link IterativeDataStream#closeWith(DataStream)}. The transformation of
-	 * this IterativeDataStream will be the iteration head. The data stream
-	 * given to the {@link IterativeDataStream#closeWith(DataStream)} method is
+	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
+	 * this IterativeStream will be the iteration head. The data stream
+	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
 	 * the data stream that will be fed back and used as the input for the
 	 * iteration head. The user can also use different feedback type than the
 	 * input of the iteration and treat the input and feedback streams as a
 	 * {@link ConnectedStreams} be calling
-	 * {@link IterativeDataStream#withFeedbackType(TypeInformation)}
+	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
 	 * <p>
 	 * A common usage pattern for streaming iterations is to use output
 	 * splitting to send a part of the closing data stream to the head. Refer to
@@ -513,7 +513,7 @@ public class DataStream<T> {
 	 * <p>
 	 * The iteration edge will be partitioned the same way as the first input of
 	 * the iteration head unless it is changed in the
-	 * {@link IterativeDataStream#closeWith(DataStream)} call.
+	 * {@link IterativeStream#closeWith(DataStream)} call.
 	 * <p>
 	 * By default a DataStream with iteration will never terminate, but the user
 	 * can use the maxWaitTime parameter to set a max waiting time for the
@@ -526,8 +526,8 @@ public class DataStream<T> {
 	 * 
 	 * @return The iterative data stream created.
 	 */
-	public IterativeDataStream<T> iterate(long maxWaitTimeMillis) {
-		return new IterativeDataStream<T>(this, maxWaitTimeMillis);
+	public IterativeStream<T> iterate(long maxWaitTimeMillis) {
+		return new IterativeStream<T>(this, maxWaitTimeMillis);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
deleted file mode 100644
index 75216ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ /dev/null
@@ -1,218 +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 org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-
-import java.util.Collection;
-
-/**
- * The iterative data stream represents the start of an iteration in a {@link DataStream}.
- * 
- * @param <T> Type of the elements in this Stream
- */
-public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, IterativeDataStream<T>> {
-
-	// We store these so that we can create a co-iteration if we need to
-	private DataStream<T> originalInput;
-	private long maxWaitTime;
-	
-	protected IterativeDataStream(DataStream<T> dataStream, long maxWaitTime) {
-		super(dataStream.getExecutionEnvironment(),
-				new FeedbackTransformation<T>(dataStream.getTransformation(), maxWaitTime));
-		this.originalInput = dataStream;
-		this.maxWaitTime = maxWaitTime;
-		setBufferTimeout(dataStream.environment.getBufferTimeout());
-	}
-
-	/**
-	 * Closes the iteration. This method defines the end of the iterative
-	 * program part that will be fed back to the start of the iteration.
-	 *
-	 * <p>
-	 * A common usage pattern for streaming iterations is to use output
-	 * splitting to send a part of the closing data stream to the head. Refer to
-	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
-	 * for more information.
-	 * 
-	 * @param feedbackStream
-	 *            {@link DataStream} that will be used as input to the iteration
-	 *            head.
-	 *
-	 * @return The feedback stream.
-	 * 
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public DataStream<T> closeWith(DataStream<T> feedbackStream) {
-
-		Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
-
-		if (!predecessors.contains(this.transformation)) {
-			throw new UnsupportedOperationException(
-					"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
-		}
-
-		((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation());
-
-		return feedbackStream;
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackTypeString
-	 *            String describing the type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStreams}.
-	 */
-	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(String feedbackTypeString) {
-		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackTypeClass
-	 *            Class of the elements in the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStreams}.
-	 */
-	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
-		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackType
-	 *            The type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStreams}.
-	 */
-	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
-		return new ConnectedIterativeDataStreams<T, F>(originalInput, feedbackType, maxWaitTime);
-	}
-	
-	/**
-	 * The {@link ConnectedIterativeDataStreams} represent a start of an
-	 * iterative part of a streaming program, where the original input of the
-	 * iteration and the feedback of the iteration are connected as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * The user can distinguish between the two inputs using co-transformation,
-	 * thus eliminating the need for mapping the inputs and outputs to a common
-	 * type.
-	 * 
-	 * @param <I>
-	 *            Type of the input of the iteration
-	 * @param <F>
-	 *            Type of the feedback of the iteration
-	 */
-	public static class ConnectedIterativeDataStreams<I, F> extends ConnectedStreams<I, F> {
-
-		private CoFeedbackTransformation<F> coFeedbackTransformation;
-
-		public ConnectedIterativeDataStreams(DataStream<I> input,
-				TypeInformation<F> feedbackType,
-				long waitTime) {
-			super(input.getExecutionEnvironment(),
-					input,
-					new DataStream<F>(input.getExecutionEnvironment(),
-							new CoFeedbackTransformation<F>(input.getParallelism(),
-									feedbackType,
-									waitTime)));
-			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecondInput().getTransformation();
-		}
-
-		/**
-		 * Closes the iteration. This method defines the end of the iterative
-		 * program part that will be fed back to the start of the iteration as
-		 * the second input in the {@link ConnectedStreams}.
-		 * 
-		 * @param feedbackStream
-		 *            {@link DataStream} that will be used as second input to
-		 *            the iteration head.
-		 * @return The feedback stream.
-		 * 
-		 */
-		@SuppressWarnings({ "rawtypes", "unchecked" })
-		public DataStream<F> closeWith(DataStream<F> feedbackStream) {
-
-			Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
-
-			if (!predecessors.contains(this.coFeedbackTransformation)) {
-				throw new UnsupportedOperationException(
-						"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
-			}
-
-			coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation());
-
-			return feedbackStream;
-		}
-		
-		private UnsupportedOperationException groupingException = new UnsupportedOperationException(
-				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
-		
-		@Override
-		public ConnectedStreams<I, F> keyBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(String field1, String field2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(String[] fields1, String[] fields2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
new file mode 100644
index 0000000..346bef9
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
@@ -0,0 +1,218 @@
+/*
+ * 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.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+
+import java.util.Collection;
+
+/**
+ * The iterative data stream represents the start of an iteration in a {@link DataStream}.
+ * 
+ * @param <T> Type of the elements in this Stream
+ */
+public class IterativeStream<T> extends SingleOutputStreamOperator<T, IterativeStream<T>> {
+
+	// We store these so that we can create a co-iteration if we need to
+	private DataStream<T> originalInput;
+	private long maxWaitTime;
+	
+	protected IterativeStream(DataStream<T> dataStream, long maxWaitTime) {
+		super(dataStream.getExecutionEnvironment(),
+				new FeedbackTransformation<T>(dataStream.getTransformation(), maxWaitTime));
+		this.originalInput = dataStream;
+		this.maxWaitTime = maxWaitTime;
+		setBufferTimeout(dataStream.environment.getBufferTimeout());
+	}
+
+	/**
+	 * Closes the iteration. This method defines the end of the iterative
+	 * program part that will be fed back to the start of the iteration.
+	 *
+	 * <p>
+	 * A common usage pattern for streaming iterations is to use output
+	 * splitting to send a part of the closing data stream to the head. Refer to
+	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
+	 * for more information.
+	 * 
+	 * @param feedbackStream
+	 *            {@link DataStream} that will be used as input to the iteration
+	 *            head.
+	 *
+	 * @return The feedback stream.
+	 * 
+	 */
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	public DataStream<T> closeWith(DataStream<T> feedbackStream) {
+
+		Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+		if (!predecessors.contains(this.transformation)) {
+			throw new UnsupportedOperationException(
+					"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
+		}
+
+		((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation());
+
+		return feedbackStream;
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackTypeString
+	 *            String describing the type information of the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(String feedbackTypeString) {
+		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackTypeClass
+	 *            Class of the elements in the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
+		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackType
+	 *            The type information of the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
+		return new ConnectedIterativeStreams<T, F>(originalInput, feedbackType, maxWaitTime);
+	}
+	
+	/**
+	 * The {@link ConnectedIterativeStreams} represent a start of an
+	 * iterative part of a streaming program, where the original input of the
+	 * iteration and the feedback of the iteration are connected as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * The user can distinguish between the two inputs using co-transformation,
+	 * thus eliminating the need for mapping the inputs and outputs to a common
+	 * type.
+	 * 
+	 * @param <I>
+	 *            Type of the input of the iteration
+	 * @param <F>
+	 *            Type of the feedback of the iteration
+	 */
+	public static class ConnectedIterativeStreams<I, F> extends ConnectedStreams<I, F> {
+
+		private CoFeedbackTransformation<F> coFeedbackTransformation;
+
+		public ConnectedIterativeStreams(DataStream<I> input,
+				TypeInformation<F> feedbackType,
+				long waitTime) {
+			super(input.getExecutionEnvironment(),
+					input,
+					new DataStream<F>(input.getExecutionEnvironment(),
+							new CoFeedbackTransformation<F>(input.getParallelism(),
+									feedbackType,
+									waitTime)));
+			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecondInput().getTransformation();
+		}
+
+		/**
+		 * Closes the iteration. This method defines the end of the iterative
+		 * program part that will be fed back to the start of the iteration as
+		 * the second input in the {@link ConnectedStreams}.
+		 * 
+		 * @param feedbackStream
+		 *            {@link DataStream} that will be used as second input to
+		 *            the iteration head.
+		 * @return The feedback stream.
+		 * 
+		 */
+		@SuppressWarnings({ "rawtypes", "unchecked" })
+		public DataStream<F> closeWith(DataStream<F> feedbackStream) {
+
+			Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+			if (!predecessors.contains(this.coFeedbackTransformation)) {
+				throw new UnsupportedOperationException(
+						"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
+			}
+
+			coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation());
+
+			return feedbackStream;
+		}
+		
+		private UnsupportedOperationException groupingException = new UnsupportedOperationException(
+				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
+		
+		@Override
+		public ConnectedStreams<I, F> keyBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(String field1, String field2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(String[] fields1, String[] fields2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
+		
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 43371b7..7bdebf8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -31,8 +31,8 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream.ConnectedIterativeDataStreams;
+import org.apache.flink.streaming.api.datastream.IterativeStream;
+import org.apache.flink.streaming.api.datastream.IterativeStream.ConnectedIterativeStreams;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -67,7 +67,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Integer> source = env.fromElements(1, 10);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
 		SingleOutputStreamOperator<Integer, ?> map1 = iter1.map(NoOpIntMap);
 		iter1.closeWith(map1).print();
 	}
@@ -80,7 +80,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
 
 		iter1.closeWith(iter1.map(NoOpIntMap));
 		iter1.closeWith(iter1.map(NoOpIntMap));
@@ -96,7 +96,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source = env.fromElements(1, 10)
 				.map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
 
 
 		iter1.closeWith(iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2));
@@ -112,7 +112,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		ConnectedIterativeDataStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
+		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
 
@@ -131,8 +131,8 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
-		IterativeDataStream<Integer> iter2 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter2 = source.iterate();
 
 
 		iter2.closeWith(iter1.map(NoOpIntMap));
@@ -150,8 +150,8 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
-		ConnectedIterativeDataStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
+		IterativeStream<Integer> iter1 = source.iterate();
+		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
 
@@ -166,7 +166,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
 
 		iter1.map(NoOpIntMap).print();
 
@@ -179,9 +179,9 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap); // for rebalance
 
-		IterativeDataStream<Integer> iter1 = source.iterate();
+		IterativeStream<Integer> iter1 = source.iterate();
 		// Calling withFeedbackType should create a new iteration
-		ConnectedIterativeDataStreams<Integer, String> iter2 = iter1.withFeedbackType(String.class);
+		ConnectedIterativeStreams<Integer, String> iter2 = iter1.withFeedbackType(String.class);
 
 		iter1.closeWith(iter1.map(NoOpIntMap)).print();
 		iter2.closeWith(iter2.map(NoOpCoMap)).print();
@@ -205,7 +205,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
 				.map(NoOpIntMap).name("ParallelizeMapRebalance");
 
-		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
+		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
 
 		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2);
 		DataStream<Integer> head2 = iter1.map(NoOpIntMap).name("IterForwardMap");
@@ -286,7 +286,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
 				.map(NoOpIntMap);
 
-		IterativeDataStream<Integer> iter1 = source1.union(source2).iterate();
+		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
 
 		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("map1");
 		DataStream<Integer> head2 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).rebalance().name(
@@ -370,7 +370,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Boolean> source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
 				.map(NoOpBoolMap).name("ParallelizeMap");
 
-		IterativeDataStream<Boolean> iteration = source.iterate(3000);
+		IterativeStream<Boolean> iteration = source.iterate(3000);
 
 		DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).map(NoOpBoolMap);
 
@@ -395,7 +395,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 				.map(NoOpStrMap).name("ParallelizeMap");
 
 
-		ConnectedIterativeDataStreams<Integer, String> coIt = env.fromElements(0, 0)
+		ConnectedIterativeStreams<Integer, String> coIt = env.fromElements(0, 0)
 				.map(NoOpIntMap).name("ParallelizeMap")
 				.iterate(2000)
 				.withFeedbackType("String");
@@ -476,7 +476,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source = env.fromElements(1, 2, 3)
 				.map(NoOpIntMap).name("ParallelizeMap");
 
-		IterativeDataStream<Integer> it = source.keyBy(key).iterate(3000);
+		IterativeStream<Integer> it = source.keyBy(key).iterate(3000);
 
 		DataStream<Integer> head = it.flatMap(new RichFlatMapFunction<Integer, Integer>() {
 
@@ -518,7 +518,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 				.map(NoOpBoolMap).name("ParallelizeMap");
 
 
-		IterativeDataStream<Boolean> iteration = source.iterate(3000);
+		IterativeStream<Boolean> iteration = source.iterate(3000);
 
 		iteration.closeWith(iteration.flatMap(new IterationHead())).addSink(new ReceiveCheckNoOpSink<Boolean>());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
index 6c439b9..d35c9bd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
@@ -29,7 +29,7 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream;
+import org.apache.flink.streaming.api.datastream.IterativeStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -115,7 +115,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		DataStream<Tuple2<Long, Tuple2<String, Long>>> sourceStream1 = env.addSource(new TupleSource()).setParallelism(1);
 
-		IterativeDataStream<Tuple2<Long, Tuple2<String, Long>>> it = sourceStream1.map(new MapFunction<Tuple2<Long, Tuple2<String, Long>>,Tuple2<Long, Tuple2<String, Long>>>(){
+		IterativeStream<Tuple2<Long, Tuple2<String, Long>>> it = sourceStream1.map(new MapFunction<Tuple2<Long, Tuple2<String, Long>>,Tuple2<Long, Tuple2<String, Long>>>(){
 
 					Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<Long, Tuple2<String, Long>>(
 							0L, new Tuple2<String, Long>("", 0L));

http://git-wip-us.apache.org/repos/asf/flink/blob/7b6e762f/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
index 8860e58..af19af7 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream;
+import org.apache.flink.streaming.api.datastream.IterativeStream;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -69,7 +69,7 @@ public class IterateExample {
 		}
 
 		// create an iterative data stream from the input with 5 second timeout
-		IterativeDataStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> it = inputStream.map(new InputMap())
+		IterativeStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> it = inputStream.map(new InputMap())
 				.iterate(5000);
 
 		// apply the step function to get the next Fibonacci number


[10/13] flink git commit: [FLINK-2550] Remove groupBy and GroupedDataStream

Posted by al...@apache.org.
[FLINK-2550] Remove groupBy and GroupedDataStream

Their functionality is subsumed by keyBy and KeyedStream


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

Branch: refs/heads/master
Commit: 9baadfe8464976b039e3f5859b910c4ad2e29ac5
Parents: 23d8e26
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Oct 1 17:56:13 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 docs/apis/streaming_guide.md                    |   6 +-
 .../api/FlinkTopologyBuilder.java               |   4 +-
 .../wordcount/BoltTokenizerWordCount.java       |   2 +-
 .../wordcount/BoltTokenizerWordCountPojo.java   |   2 +-
 .../BoltTokenizerWordCountWithNames.java        |   2 +-
 .../wordcount/SpoutSourceWordCount.java         |   2 +-
 .../examples/java8/wordcount/WordCount.java     |   2 +-
 .../main/java/SocketTextStreamWordCount.java    |   2 +-
 .../main/scala/SocketTextStreamWordCount.scala  |   2 +-
 .../connectors/twitter/TwitterTopology.java     |   2 +-
 .../api/datastream/ConnectedStreams.java        |  40 +--
 .../streaming/api/datastream/DataStream.java    |  62 ----
 .../api/datastream/DiscretizedStream.java       |  20 +-
 .../api/datastream/GroupedDataStream.java       | 329 -------------------
 .../api/datastream/IterativeDataStream.java     |  19 +-
 .../streaming/api/datastream/KeyedStream.java   | 297 ++++++++++++++++-
 .../api/datastream/WindowedDataStream.java      |  62 ++--
 .../datastream/temporal/StreamJoinOperator.java |   8 +-
 .../flink/streaming/api/CoStreamTest.java       |   4 +-
 .../flink/streaming/api/DataStreamTest.java     |  72 ++--
 .../apache/flink/streaming/api/IterateTest.java |   6 +-
 .../streaming/api/StreamingOperatorsITCase.java |   4 +-
 .../api/complex/ComplexIntegrationTest.java     |   4 +-
 .../api/operators/co/SelfConnectionTest.java    |   4 +-
 .../windowing/ParallelMergeITCase.java          |   2 +-
 .../operators/windowing/WindowingITCase.java    |  14 +-
 .../api/outputformat/CsvOutputFormatITCase.java |   2 +-
 .../outputformat/SocketOutputFormatITCase.java  |   2 +-
 .../outputformat/TextOutputFormatITCase.java    |   2 +-
 .../socket/SocketTextStreamWordCount.java       |   2 +-
 .../examples/twitter/TwitterStream.java         |   2 +-
 .../examples/windowing/SessionWindowing.java    |   2 +-
 .../examples/windowing/TopSpeedWindowing.java   |   2 +-
 .../examples/windowing/WindowWordCount.java     |   2 +-
 .../examples/wordcount/PojoExample.java         |   2 +-
 .../streaming/examples/wordcount/WordCount.java |   2 +-
 .../socket/SocketTextStreamWordCount.scala      |   2 +-
 .../examples/windowing/TopSpeedWindowing.scala  |   2 +-
 .../streaming/api/scala/ConnectedStreams.scala  |  20 +-
 .../flink/streaming/api/scala/DataStream.scala  |  37 +--
 .../streaming/api/scala/GroupedDataStream.scala | 196 -----------
 .../flink/streaming/api/scala/KeyedStream.scala | 195 +++++++++++
 .../api/scala/StreamJoinOperator.scala          |   4 +-
 .../api/scala/WindowedDataStream.scala          |  10 +-
 .../flink/streaming/api/scala/package.scala     |  10 +-
 .../streaming/api/scala/DataStreamTest.scala    |  26 +-
 .../api/scala/OutputFormatTestPrograms.scala    |   6 +-
 .../api/scala/StreamingOperatorsITCase.scala    |   2 +-
 .../CoStreamCheckpointingITCase.java            |   2 +-
 .../PartitionedStateCheckpointingITCase.java    |   2 +-
 .../StreamCheckpointNotifierITCase.java         |   2 +-
 .../StreamCheckpointingITCase.java              |   2 +-
 .../test/classloading/jar/StreamingProgram.java |   2 +-
 53 files changed, 690 insertions(+), 823 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/docs/apis/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md
index c437114..3f5a98f 100644
--- a/docs/apis/streaming_guide.md
+++ b/docs/apis/streaming_guide.md
@@ -627,7 +627,7 @@ dataStream.filter{ _ != 0 }
 	<br/>
 	A map that produces a rolling average per key:</p>
 {% highlight scala %}
-dataStream.keyBy(..).mapWithState((in, state: Option[(Long, Int)]) => state match {
+dataStream.groupBy(..).mapWithState((in, state: Option[(Long, Int)]) => state match {
 	case Some((sum, count)) => ((sum + in)/(count + 1), Some((sum + in, count + 1)))
 	case None => (in, Some((in, 1)))
 })
@@ -713,7 +713,7 @@ dataStream.union(otherStream1, otherStream2, …)
 
 ### Grouped operators
 
-Some transformations require that the elements of a `DataStream` are grouped on some key. The user can create a `GroupedDataStream` by calling the `groupBy(key)` method of a non-grouped `DataStream`. 
+Some transformations require that the elements of a `DataStream` are grouped on some key. The user can create a `GroupedDataStream` by calling the `groupBy(key)` method of a non-grouped `DataStream`.
 Keys can be of three types: field positions (applicable for tuple/array types), field expressions (applicable for pojo types), KeySelector instances.
 
 Aggregation or reduce operators called on `GroupedDataStream`s produce elements on a per group basis.
@@ -1313,7 +1313,7 @@ Checkpointing of the states needs to be enabled from the `StreamExecutionEnviron
 
 Operator states can be accessed from the `RuntimeContext` using the `getOperatorState(“name”, defaultValue, partitioned)` method so it is only accessible in `RichFunction`s. A recommended usage pattern is to retrieve the operator state in the `open(…)` method of the operator and set it as a field in the operator instance for runtime usage. Multiple `OperatorState`s can be used simultaneously by the same operator by using different names to identify them.
 
-Partitioned operator state works only on `KeyedDataStreams`. A `KeyedDataStream` can be created from `DataStream` using the `keyBy` or `groupBy` methods. The `keyBy` method simply takes a `KeySelector` to derive the keys by which the operator state will be partitioned, however, it does not affect the actual partitioning of the `DataStream` records. If data partitioning is also desired then the `groupBy`  method should be used instead to create a `GroupedDataStream` which is a subtype of `KeyedDataStream`. Mind that `KeyedDataStreams` do not support repartitioning (e.g. `shuffle(), forward(), groupBy(...)`).
+Partitioned operator state works only on `KeyedDataStreams`. A `KeyedDataStream` can be created from `DataStream` using the `groupBy` or `groupBy` methods. The `groupBy` method simply takes a `KeySelector` to derive the keys by which the operator state will be partitioned, however, it does not affect the actual partitioning of the `DataStream` records. If data partitioning is also desired then the `groupBy`  method should be used instead to create a `GroupedDataStream` which is a subtype of `KeyedDataStream`. Mind that `KeyedDataStreams` do not support repartitioning (e.g. `shuffle(), forward(), groupBy(...)`).
 
 By default operator states are checkpointed using default java serialization thus they need to be `Serializable`. The user can gain more control over the state checkpoint mechanism by passing a `StateCheckpointer` instance when retrieving the `OperatorState` from the `RuntimeContext`. The `StateCheckpointer` allows custom implementations for the checkpointing logic for increased efficiency and to store arbitrary non-serializable states.
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
index e4d880f..e2d819c 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
@@ -195,12 +195,12 @@ public class FlinkTopologyBuilder {
 								if (fields.size() > 0) {
 									FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId);
 									if (producer.size() == 1) {
-										inputStream = inputStream.groupBy(prodDeclarer
+										inputStream = inputStream.keyBy(prodDeclarer
 												.getGroupingFieldIndexes(inputStreamId,
 														grouping.get_fields()));
 									} else {
 										inputStream = inputStream
-												.groupBy(new SplitStreamTypeKeySelector(
+												.keyBy(new SplitStreamTypeKeySelector(
 														prodDeclarer.getGroupingFieldIndexes(
 																inputStreamId,
 																grouping.get_fields())));

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
index eab58f5..6f7b6fb 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
@@ -69,7 +69,7 @@ public class BoltTokenizerWordCount {
 						new StormBoltWrapper<String, Tuple2<String, Integer>>(new StormBoltTokenizer()))
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
index 20e69db..300f5bc 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
@@ -77,7 +77,7 @@ public class BoltTokenizerWordCountPojo {
 								new StormBoltTokenizerByName()))
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
index e233da1..ed01181 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
@@ -80,7 +80,7 @@ public class BoltTokenizerWordCountWithNames {
 								new StormBoltTokenizerByName(), new Fields("sentence")))
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
index cbd054b..21d7811 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
@@ -70,7 +70,7 @@ public class SpoutSourceWordCount {
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				text.flatMap(new Tokenizer())
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0).sum(1);
+				.keyBy(0).sum(1);
 
 		// emit result
 		if (fileOutput) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java b/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java
index c1d6042..41a674f 100644
--- a/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java
+++ b/flink-java8/src/main/java/org/apache/flink/streaming/examples/java8/wordcount/WordCount.java
@@ -73,7 +73,7 @@ public class WordCount {
 					.forEach(t -> out.collect(new Tuple2<>(t, 1)));
 				})
 				// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0)
+				.keyBy(0)
 				.sum(1);
 
 		// emit result

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java
index 3888d7b..10d8044 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java
@@ -76,7 +76,7 @@ public class SocketTextStreamWordCount {
 		// split up the lines in pairs (2-tuples) containing: (word,1)
 		text.flatMap(new LineSplitter())
 		// group by the tuple field "0" and sum up tuple field "1"
-				.groupBy(0)
+				.keyBy(0)
 				.sum(1);
 
 		counts.print();

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala
index 63d840d..9bc85ea 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala
@@ -59,7 +59,7 @@ object SocketTextStreamWordCount {
     val text = env.socketTextStream(hostName, port)
     val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
       .map { (_, 1) }
-      .groupBy(0)
+      .keyBy(0)
       .sum(1)
 
     counts print

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
index d5e8c41..b1fc92c 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
@@ -82,7 +82,7 @@ public class TwitterTopology {
 						return new Tuple2<String, Integer>(value, 1);
 					}
 				})
-				.groupBy(0)
+				.keyBy(0)
 				.sum(1);
 
 		dataStream.print();

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
index 2447c1e..4074a1d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
@@ -96,7 +96,7 @@ public class ConnectedStreams<IN1, IN2> {
 	}
 
 	/**
-	 * GroupBy operation for connected data stream. Groups the elements of
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
 	 * input1 and input2 according to keyPosition1 and keyPosition2.
 	 *
 	 * @param keyPosition1
@@ -107,13 +107,13 @@ public class ConnectedStreams<IN1, IN2> {
 	 *            second input stream.
 	 * @return The grouped {@link ConnectedStreams}
 	 */
-	public ConnectedStreams<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
-		return new ConnectedStreams<>(this.environment, inputStream1.groupBy(keyPosition1),
-				inputStream2.groupBy(keyPosition2));
+	public ConnectedStreams<IN1, IN2> keyBy(int keyPosition1, int keyPosition2) {
+		return new ConnectedStreams<>(this.environment, inputStream1.keyBy(keyPosition1),
+				inputStream2.keyBy(keyPosition2));
 	}
 
 	/**
-	 * GroupBy operation for connected data stream. Groups the elements of
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
 	 * input1 and input2 according to keyPositions1 and keyPositions2.
 	 *
 	 * @param keyPositions1
@@ -122,13 +122,13 @@ public class ConnectedStreams<IN1, IN2> {
 	 *            The fields used to group the second input stream.
 	 * @return The grouped {@link ConnectedStreams}
 	 */
-	public ConnectedStreams<IN1, IN2> groupBy(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedStreams<>(environment, inputStream1.groupBy(keyPositions1),
-				inputStream2.groupBy(keyPositions2));
+	public ConnectedStreams<IN1, IN2> keyBy(int[] keyPositions1, int[] keyPositions2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(keyPositions1),
+				inputStream2.keyBy(keyPositions2));
 	}
 
 	/**
-	 * GroupBy operation for connected data stream using key expressions. Groups
+	 * KeyBy operation for connected data stream using key expressions. Assigns keys to
 	 * the elements of input1 and input2 according to field1 and field2. A field
 	 * expression is either the name of a public field or a getter method with
 	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
@@ -140,13 +140,13 @@ public class ConnectedStreams<IN1, IN2> {
 	 *            The grouping expression for the second input
 	 * @return The grouped {@link ConnectedStreams}
 	 */
-	public ConnectedStreams<IN1, IN2> groupBy(String field1, String field2) {
-		return new ConnectedStreams<>(environment, inputStream1.groupBy(field1),
-				inputStream2.groupBy(field2));
+	public ConnectedStreams<IN1, IN2> keyBy(String field1, String field2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(field1),
+				inputStream2.keyBy(field2));
 	}
 
 	/**
-	 * GroupBy operation for connected data stream using key expressions. Groups
+	 * KeyBy operation for connected data stream using key expressions.
 	 * the elements of input1 and input2 according to fields1 and fields2. A
 	 * field expression is either the name of a public field or a getter method
 	 * with parentheses of the {@link DataStream}S underlying type. A dot can be
@@ -159,13 +159,13 @@ public class ConnectedStreams<IN1, IN2> {
 	 *            The grouping expressions for the second input
 	 * @return The grouped {@link ConnectedStreams}
 	 */
-	public ConnectedStreams<IN1, IN2> groupBy(String[] fields1, String[] fields2) {
-		return new ConnectedStreams<>(environment, inputStream1.groupBy(fields1),
-				inputStream2.groupBy(fields2));
+	public ConnectedStreams<IN1, IN2> keyBy(String[] fields1, String[] fields2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(fields1),
+				inputStream2.keyBy(fields2));
 	}
 
 	/**
-	 * GroupBy operation for connected data stream. Groups the elements of
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
 	 * input1 and input2 using keySelector1 and keySelector2.
 	 *
 	 * @param keySelector1
@@ -174,9 +174,9 @@ public class ConnectedStreams<IN1, IN2> {
 	 *            The {@link KeySelector} used for grouping the second input
 	 * @return The partitioned {@link ConnectedStreams}
 	 */
-	public ConnectedStreams<IN1, IN2> groupBy(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedStreams<>(environment, inputStream1.groupBy(keySelector1),
-				inputStream2.groupBy(keySelector2));
+	public ConnectedStreams<IN1, IN2> keyBy(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(keySelector1),
+				inputStream2.keyBy(keySelector2));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 3389016..6d88416 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -278,68 +278,6 @@ public class DataStream<T> {
 		return new KeyedStream<T, Tuple>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
 				getType(), getExecutionConfig())));
 	}
-	
-	/**
-	 * Partitions the operator state of a {@link DataStream} by the given key positions. 
-	 * Mind that keyBy does not affect the partitioning of the {@link DataStream}
-	 * but only the way explicit state is partitioned among parallel instances.
-	 * 
-	 * @param fields
-	 *            The position of the fields on which the states of the {@link DataStream}
-	 *            will be partitioned.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
-	 */
-	public GroupedDataStream<T, Tuple> groupBy(int... fields) {
-		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return groupBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
-		} else {
-			return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
-		}
-	}
-
-	/**
-	 * Groups a {@link DataStream} using field expressions. A field expression
-	 * is either the name of a public field or a getter method with parentheses
-	 * of the {@link DataStream}S underlying type. A dot can be used to drill
-	 * down into objects, as in {@code "field1.getInnerField2()" }. This method
-	 * returns an {@link GroupedDataStream}.
-	 *
-	 * <p>
-	 * This operator also affects the
-	 * partitioning of the stream, by forcing values with the same key to go to
-	 * the same processing instance.
-	 * 
-	 * @param fields
-	 *            One or more field expressions on which the DataStream will be
-	 *            grouped.
-	 * @return The grouped {@link DataStream}
-	 **/
-	public GroupedDataStream<T, Tuple> groupBy(String... fields) {
-		return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
-	}
-
-	/**
-	 * Groups the elements of a {@link DataStream} by the key extracted by the
-	 * {@link KeySelector} to be used with grouped operators like
-	 * {@link GroupedDataStream#reduce(org.apache.flink.api.common.functions.ReduceFunction)}.
-	 *
-	 * <p/>
-	 * This operator also affects the partitioning of the stream, by forcing
-	 * values with the same key to go to the same processing instance.
-	 * 
-	 * @param keySelector
-	 *            The {@link KeySelector} that will be used to extract keys for
-	 *            the values
-	 * @return The grouped {@link DataStream}
-	 */
-	public <K> GroupedDataStream<T, K> groupBy(KeySelector<T, K> keySelector) {
-		return new GroupedDataStream<T, K>(this, clean(keySelector));
-	}
-
-	private GroupedDataStream<T, Tuple> groupBy(Keys<T> keys) {
-		return new GroupedDataStream<T, Tuple>(this, 
-				clean(KeySelectorUtil.getSelectorForKeys(keys, getType(), getExecutionConfig())));
-	}
 
 	/**
 	 * Sets the partitioning of the {@link DataStream} so that the output is

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
index 5893295..18c2cee 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
@@ -66,10 +66,10 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 	protected boolean isPartitioned = false;
 
 	protected DiscretizedStream(SingleOutputStreamOperator<StreamWindow<OUT>, ?> discretizedStream,
-			KeySelector<OUT, ?> groupByKey, WindowTransformation tranformation,
+			KeySelector<OUT, ?> keyByKey, WindowTransformation tranformation,
 			boolean isPartitioned) {
 		super();
-		this.groupByKey = groupByKey;
+		this.keyByKey = keyByKey;
 		this.discretizedStream = discretizedStream;
 		this.transformation = tranformation;
 		this.isPartitioned = isPartitioned;
@@ -151,8 +151,8 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 				: new ParallelMerge<OUT>(reduceFunction);
 
 		return reduced.discretizedStream
-				.groupBy(new WindowKey<OUT>())
-				.connect(numOfParts.groupBy(0))
+				.keyBy(new WindowKey<OUT>())
+				.connect(numOfParts.keyBy(0))
 				.transform(
 						"CoFlatMap",
 						reduced.discretizedStream.getType(),
@@ -218,9 +218,9 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 
 		if (isGrouped()) {
 			DiscretizedStream<OUT> out = transform(transformation, "Window partitioner", getType(),
-					new WindowPartitioner<OUT>(groupByKey)).setParallelism(parallelism);
+					new WindowPartitioner<OUT>(keyByKey)).setParallelism(parallelism);
 
-			out.groupByKey = null;
+			out.keyByKey = null;
 			out.isPartitioned = true;
 
 			return out;
@@ -247,7 +247,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 		// Only merge partitioned streams
 		if (isPartitioned) {
 			return wrap(
-					discretizedStream.groupBy(new WindowKey<OUT>()).transform("Window Merger",
+					discretizedStream.keyBy(new WindowKey<OUT>()).transform("Window Merger",
 							type, new WindowMerger<OUT>()).setParallelism(discretizedStream.getParallelism()), false);
 		} else {
 			return this;
@@ -258,14 +258,14 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 	@SuppressWarnings("unchecked")
 	private <R> DiscretizedStream<R> wrap(SingleOutputStreamOperator<StreamWindow<R>, ?> stream,
 			boolean isPartitioned) {
-		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.groupByKey,
+		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.keyByKey,
 				transformation, isPartitioned);
 	}
 
 	@SuppressWarnings("unchecked")
 	private <R> DiscretizedStream<R> wrap(SingleOutputStreamOperator<StreamWindow<R>, ?> stream,
 			WindowTransformation transformation) {
-		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.groupByKey,
+		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.keyByKey,
 				transformation, isPartitioned);
 	}
 
@@ -329,7 +329,7 @@ public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
 	}
 
 	protected DiscretizedStream<OUT> copy() {
-		return new DiscretizedStream<OUT>(discretizedStream, groupByKey, transformation, isPartitioned);
+		return new DiscretizedStream<OUT>(discretizedStream, keyByKey, transformation, isPartitioned);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
deleted file mode 100644
index ebaeb56..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
+++ /dev/null
@@ -1,329 +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 org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.operators.StreamGroupedFold;
-import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
-
-/**
- * A GroupedDataStream represents a {@link DataStream} which has been
- * partitioned by the given {@link KeySelector}. Operators like {@link #reduce},
- * {@link #fold} etc. can be applied on the {@link GroupedDataStream} to
- * get additional functionality by the grouping.
- *
- * @param <T> The type of the elements in the Grouped Stream.
- * @param <KEY> The type of the key in the Keyed Stream.
- */
-public class GroupedDataStream<T, KEY> extends KeyedStream<T, KEY> {
-
-	/**
-	 * Creates a new {@link GroupedDataStream}, group inclusion is determined using
-	 * a {@link KeySelector} on the elements of the {@link DataStream}.
-	 *
-	 * @param dataStream Base stream of data
-	 * @param keySelector Function for determining group inclusion
-	 */
-	public GroupedDataStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
-		super(dataStream, keySelector);
-	}
-
-
-	/**
-	 * 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.
-	 * 
-	 * @param reducer
-	 *            The {@link ReduceFunction} that will be called for every
-	 *            element of the input values with the same key.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
-		return transform("Grouped Reduce", getType(), new StreamGroupedReduce<T>(
-				clean(reducer), keySelector));
-	}
-
-	/**
-	 * Applies a fold transformation on the grouped data stream grouped on by
-	 * the given key position. The {@link FoldFunction} will receive input
-	 * values based on the key value. Only input values with the same key will
-	 * go to the same folder.
-	 * 
-	 * @param folder
-	 *            The {@link FoldFunction} that will be called for every element
-	 *            of the input values with the same key.
-	 * @param initialValue
-	 *            The initialValue passed to the folders for each key.
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> folder) {
-
-		TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(clean(folder), getType(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Grouped Fold", outType, new StreamGroupedFold<T, R>(clean(folder),
-				keySelector, initialValue));
-	}
-
-	/**
-	 * Applies an aggregation that gives a rolling sum of the data stream at the
-	 * given position grouped by the given key. An independent aggregate is kept
-	 * per key.
-	 * 
-	 * @param positionToSum
-	 *            The position in the data point to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
-		return aggregate(new SumAggregator<T>(positionToSum, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current sum of the pojo data
-	 * stream at the given field expressionby the given key. An independent
-	 * aggregate is kept per key. A field expression is either the name of a
-	 * public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(String field) {
-		return aggregate(new SumAggregator<T>(field, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum of the data
-	 * stream at the given position by the given key. An independent aggregate
-	 * is kept per key.
-	 * 
-	 * @param positionToMin
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
-		return aggregate(new ComparableAggregator<T>(positionToMin, getType(), AggregationType.MIN,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum of the pojo
-	 * data stream at the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(String field) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MIN,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the current maximum of the data stream
-	 * at the given position by the given key. An independent aggregate is kept
-	 * per key.
-	 * 
-	 * @param positionToMax
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
-		return aggregate(new ComparableAggregator<T>(positionToMax, getType(), AggregationType.MAX,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current maximum of the pojo
-	 * data stream at the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(String field) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MAX,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum element of the
-	 * pojo data stream by the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator(field, getType(), AggregationType.MINBY,
-				first, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current maximum element of the
-	 * pojo data stream by the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MAXBY,
-				first, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns the first one by default.
-	 * 
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns the first one by default.
-	 * 
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns either the first or last one,
-	 * depending on the parameter set.
-	 * 
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            minimal value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
-		return aggregate(new ComparableAggregator<T>(positionToMinBy, getType(), AggregationType.MINBY, first,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns the first one by default.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns the first one by default.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns either the first or last one,
-	 * depending on the parameter set.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize.
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            maximum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
-		return aggregate(new ComparableAggregator<T>(positionToMaxBy, getType(), AggregationType.MAXBY, first,
-				getExecutionConfig()));
-	}
-
-	protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
-		StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(clean(aggregate), keySelector);
-		return transform("Grouped Aggregation", getType(), operator);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index 2fe3848..75216ec 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -188,20 +188,17 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
 				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
 		
 		@Override
-		public ConnectedStreams<I, F> groupBy(int keyPosition1, int keyPosition2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> groupBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
-		
+		public ConnectedStreams<I, F> keyBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+
 		@Override
-		public ConnectedStreams<I, F> groupBy(String field1, String field2) {throw groupingException;}
-		
+		public ConnectedStreams<I, F> keyBy(String field1, String field2) {throw groupingException;}
+
 		@Override
-		public ConnectedStreams<I, F> groupBy(String[] fields1, String[] fields2) {throw groupingException;}
-		
+		public ConnectedStreams<I, F> keyBy(String[] fields1, String[] fields2) {throw groupingException;}
+
 		@Override
-		public ConnectedStreams<I, F> groupBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
-		
+		public ConnectedStreams<I, F> keyBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
+
 		@Override
 		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
index b3cfb55..265886b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -17,10 +17,19 @@
 
 package org.apache.flink.streaming.api.datastream;
 
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamGroupedFold;
+import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.PartitionTransformation;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
@@ -39,9 +48,12 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
  * A {@code KeyedStream} represents a {@link DataStream} on which operator state is
  * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a
  * {@code DataStream} are also possible on a {@code KeyedStream}, with the exception of
- * partitioning methods such as shuffle, forward and groupBy.
- * 
- * 
+ * partitioning methods such as shuffle, forward and keyBy.
+ *
+ * <p>
+ * Reduce-style operations, such as {@link #reduce}, {@link #sum} and {@link #fold} work on elements
+ * that have the same key.
+ *
  * @param <T> The type of the elements in the Keyed Stream.
  * @param <KEY> The type of the key in the Keyed Stream.
  */
@@ -59,7 +71,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 	 *            Function for determining state partitions
 	 */
 	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
-		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<T>(dataStream.getTransformation(), new HashPartitioner<T>(keySelector)));
+		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<>(dataStream.getTransformation(), new HashPartitioner<>(keySelector)));
 		this.keySelector = keySelector;
 	}
 
@@ -157,4 +169,281 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 	public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
 		return new WindowedStream<>(this, assigner);
 	}
+
+	// ------------------------------------------------------------------------
+	//  Non-Windowed aggregation operations
+	// ------------------------------------------------------------------------
+
+	/**
+	 * 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.
+	 *
+	 * @param reducer
+	 *            The {@link ReduceFunction} that will be called for every
+	 *            element of the input values with the same key.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
+		return transform("Keyed Reduce", getType(), new StreamGroupedReduce<>(clean(reducer), keySelector));
+	}
+
+	/**
+	 * Applies a fold transformation on the grouped data stream grouped on by
+	 * the given key position. The {@link FoldFunction} will receive input
+	 * values based on the key value. Only input values with the same key will
+	 * go to the same folder.
+	 *
+	 * @param folder
+	 *            The {@link FoldFunction} that will be called for every element
+	 *            of the input values with the same key.
+	 * @param initialValue
+	 *            The initialValue passed to the folders for each key.
+	 * @return The transformed DataStream.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> folder) {
+
+		TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(clean(folder), getType(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder),
+				keySelector, initialValue));
+	}
+
+	/**
+	 * Applies an aggregation that gives a rolling sum of the data stream at the
+	 * given position grouped by the given key. An independent aggregate is kept
+	 * per key.
+	 *
+	 * @param positionToSum
+	 *            The position in the data point to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, getType(), getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current sum of the pojo data
+	 * stream at the given field expressionby the given key. An independent
+	 * aggregate is kept per key. A field expression is either the name of a
+	 * public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
+		return aggregate(new SumAggregator<>(field, getType(), getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum of the data
+	 * stream at the given position by the given key. An independent aggregate
+	 * is kept per key.
+	 *
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, getType(), AggregationFunction.AggregationType.MIN,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum of the pojo
+	 * data stream at the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MIN,
+				false, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the current maximum of the data stream
+	 * at the given position by the given key. An independent aggregate is kept
+	 * per key.
+	 *
+	 * @param positionToMax
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, getType(), AggregationFunction.AggregationType.MAX,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current maximum of the pojo
+	 * data stream at the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAX,
+				false, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum element of the
+	 * pojo data stream by the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @param first
+	 *            If True then in case of field equality the first object will
+	 *            be returned
+	 * @return The transformed DataStream.
+	 */
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator(field, getType(), AggregationFunction.AggregationType.MINBY,
+				first, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current maximum element of the
+	 * pojo data stream by the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @param first
+	 *            If True then in case of field equality the first object will
+	 *            be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAXBY,
+				first, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns either the first or last one,
+	 * depending on the parameter set.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            minimal value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<T>(positionToMinBy, getType(), AggregationFunction.AggregationType.MINBY, first,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns either the first or last one,
+	 * depending on the parameter set.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize.
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, getType(), AggregationFunction.AggregationType.MAXBY, first,
+				getExecutionConfig()));
+	}
+
+	protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
+		StreamGroupedReduce<T> operator = new StreamGroupedReduce<>(clean(aggregate), keySelector);
+		return transform("Keyed Aggregation", getType(), operator);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
index ef6f53b..c1c5f6d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
@@ -89,7 +89,7 @@ public class WindowedDataStream<T> {
 	protected boolean isLocal = false;
 
 	protected KeySelector<T, ?> discretizerKey;
-	protected KeySelector<T, ?> groupByKey;
+	protected KeySelector<T, ?> keyByKey;
 
 	protected WindowingHelper<T> triggerHelper;
 	protected WindowingHelper<T> evictionHelper;
@@ -101,8 +101,8 @@ public class WindowedDataStream<T> {
 		this.dataStream = dataStream;
 		this.triggerHelper = policyHelper;
 
-		if (dataStream instanceof GroupedDataStream) {
-			this.discretizerKey = ((GroupedDataStream<T, ?>) dataStream).keySelector;
+		if (dataStream instanceof KeyedStream) {
+			this.discretizerKey = ((KeyedStream<T, ?>) dataStream).keySelector;
 		}
 	}
 
@@ -113,15 +113,15 @@ public class WindowedDataStream<T> {
 		this.userTrigger = trigger;
 		this.userEvicter = evicter;
 
-		if (dataStream instanceof GroupedDataStream) {
-			this.discretizerKey = ((GroupedDataStream<T, ?>) dataStream).keySelector;
+		if (dataStream instanceof KeyedStream) {
+			this.discretizerKey = ((KeyedStream<T, ?>) dataStream).keySelector;
 		}
 	}
 
 	protected WindowedDataStream(WindowedDataStream<T> windowedDataStream) {
 		this.dataStream = windowedDataStream.dataStream;
 		this.discretizerKey = windowedDataStream.discretizerKey;
-		this.groupByKey = windowedDataStream.groupByKey;
+		this.keyByKey = windowedDataStream.keyByKey;
 		this.triggerHelper = windowedDataStream.triggerHelper;
 		this.evictionHelper = windowedDataStream.evictionHelper;
 		this.userTrigger = windowedDataStream.userTrigger;
@@ -170,11 +170,11 @@ public class WindowedDataStream<T> {
 	 *            The position of the fields to group by.
 	 * @return The grouped {@link WindowedDataStream}
 	 */
-	public WindowedDataStream<T> groupBy(int... fields) {
+	public WindowedDataStream<T> keyBy(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return groupBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
+			return keyBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
 		} else {
-			return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
+			return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
 		}
 	}
 
@@ -194,8 +194,8 @@ public class WindowedDataStream<T> {
 	 *            The fields to group by
 	 * @return The grouped {@link WindowedDataStream}
 	 */
-	public WindowedDataStream<T> groupBy(String... fields) {
-		return groupBy(new Keys.ExpressionKeys<T>(fields, getType()));
+	public WindowedDataStream<T> keyBy(String... fields) {
+		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
 	}
 
 	/**
@@ -210,14 +210,14 @@ public class WindowedDataStream<T> {
 	 *            The keySelector used to extract the key for grouping.
 	 * @return The grouped {@link WindowedDataStream}
 	 */
-	public WindowedDataStream<T> groupBy(KeySelector<T, ?> keySelector) {
+	public WindowedDataStream<T> keyBy(KeySelector<T, ?> keySelector) {
 		WindowedDataStream<T> ret = this.copy();
-		ret.groupByKey = keySelector;
+		ret.keyByKey = keySelector;
 		return ret;
 	}
 
-	private WindowedDataStream<T> groupBy(Keys<T> keys) {
-		return groupBy(clean(KeySelectorUtil.getSelectorForKeys(keys, getType(),
+	private WindowedDataStream<T> keyBy(Keys<T> keys) {
+		return keyBy(clean(KeySelectorUtil.getSelectorForKeys(keys, getType(),
 				getExecutionConfig())));
 	}
 
@@ -398,7 +398,7 @@ public class WindowedDataStream<T> {
 				.setParallelism(parallelism)
 				.transform(windowBuffer.getClass().getSimpleName(),
 						new StreamWindowTypeInfo<T>(getType()), bufferOperator)
-				.setParallelism(parallelism), groupByKey, transformation, false);
+				.setParallelism(parallelism), keyByKey, transformation, false);
 
 	}
 
@@ -442,8 +442,8 @@ public class WindowedDataStream<T> {
 		// If there is a groupby for the reduce operation we apply it before the
 		// discretizers, because we will forward everything afterwards to
 		// exploit task chaining
-		if (groupByKey != null) {
-			dataStream = dataStream.groupBy(groupByKey);
+		if (keyByKey != null) {
+			dataStream = dataStream.keyBy(keyByKey);
 		}
 
 		// We discretize the stream and call the timeReduce function of the
@@ -502,28 +502,28 @@ public class WindowedDataStream<T> {
 		if (transformation == WindowTransformation.REDUCEWINDOW) {
 			if (WindowUtils.isTumblingPolicy(trigger, eviction)) {
 				if (eviction instanceof KeepAllEvictionPolicy) {
-					if (groupByKey == null) {
+					if (keyByKey == null) {
 						return new TumblingPreReducer<T>(
 								(ReduceFunction<T>) transformation.getUDF(), getType()
 										.createSerializer(getExecutionConfig())).noEvict();
 					} else {
 						return new TumblingGroupedPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), groupByKey,
+								(ReduceFunction<T>) transformation.getUDF(), keyByKey,
 								getType().createSerializer(getExecutionConfig())).noEvict();
 					}
 				} else {
-					if (groupByKey == null) {
+					if (keyByKey == null) {
 						return new TumblingPreReducer<T>(
 								(ReduceFunction<T>) transformation.getUDF(), getType()
 										.createSerializer(getExecutionConfig()));
 					} else {
 						return new TumblingGroupedPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), groupByKey,
+								(ReduceFunction<T>) transformation.getUDF(), keyByKey,
 								getType().createSerializer(getExecutionConfig()));
 					}
 				}
 			} else if (WindowUtils.isSlidingCountPolicy(trigger, eviction)) {
-				if (groupByKey == null) {
+				if (keyByKey == null) {
 					return new SlidingCountPreReducer<T>(
 							clean((ReduceFunction<T>) transformation.getUDF()), dataStream
 									.getType().createSerializer(getExecutionConfig()),
@@ -532,13 +532,13 @@ public class WindowedDataStream<T> {
 				} else {
 					return new SlidingCountGroupedPreReducer<T>(
 							clean((ReduceFunction<T>) transformation.getUDF()), dataStream
-									.getType().createSerializer(getExecutionConfig()), groupByKey,
+									.getType().createSerializer(getExecutionConfig()), keyByKey,
 							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
 							((CountTriggerPolicy<?>) trigger).getStart());
 				}
 
 			} else if (WindowUtils.isSlidingTimePolicy(trigger, eviction)) {
-				if (groupByKey == null) {
+				if (keyByKey == null) {
 					return new SlidingTimePreReducer<T>(
 							(ReduceFunction<T>) transformation.getUDF(), dataStream.getType()
 									.createSerializer(getExecutionConfig()),
@@ -547,25 +547,25 @@ public class WindowedDataStream<T> {
 				} else {
 					return new SlidingTimeGroupedPreReducer<T>(
 							(ReduceFunction<T>) transformation.getUDF(), dataStream.getType()
-									.createSerializer(getExecutionConfig()), groupByKey,
+									.createSerializer(getExecutionConfig()), keyByKey,
 							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
 							WindowUtils.getTimeStampWrapper(trigger));
 				}
 
 			} else if (WindowUtils.isJumpingCountPolicy(trigger, eviction)) {
-				if (groupByKey == null) {
+				if (keyByKey == null) {
 					return new JumpingCountPreReducer<T>(
 							(ReduceFunction<T>) transformation.getUDF(), getType()
 									.createSerializer(getExecutionConfig()),
 							WindowUtils.getSlideSize(trigger) - WindowUtils.getWindowSize(eviction));
 				} else {
 					return new JumpingCountGroupedPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), groupByKey, getType()
+							(ReduceFunction<T>) transformation.getUDF(), keyByKey, getType()
 									.createSerializer(getExecutionConfig()),
 							WindowUtils.getSlideSize(trigger) - WindowUtils.getWindowSize(eviction));
 				}
 			} else if (WindowUtils.isJumpingTimePolicy(trigger, eviction)) {
-				if (groupByKey == null) {
+				if (keyByKey == null) {
 					return new JumpingTimePreReducer<T>(
 							(ReduceFunction<T>) transformation.getUDF(), getType()
 									.createSerializer(getExecutionConfig()),
@@ -573,7 +573,7 @@ public class WindowedDataStream<T> {
 							WindowUtils.getTimeStampWrapper(trigger));
 				} else {
 					return new JumpingTimeGroupedPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), groupByKey, getType()
+							(ReduceFunction<T>) transformation.getUDF(), keyByKey, getType()
 									.createSerializer(getExecutionConfig()),
 							WindowUtils.getSlideSize(trigger), WindowUtils.getWindowSize(eviction),
 							WindowUtils.getTimeStampWrapper(trigger));
@@ -845,7 +845,7 @@ public class WindowedDataStream<T> {
 	}
 
 	protected boolean isGrouped() {
-		return groupByKey != null;
+		return keyByKey != null;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
index 999d197..4a5622d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
@@ -211,8 +211,8 @@ public class StreamJoinOperator<I1, I2> extends
 //					op.input1.getType(), op.input2.getType());
 
 //			return new JoinedStream<I1, I2, Tuple2<I1, I2>>(this, op.input1
-//					.groupBy(keys1)
-//					.connect(op.input2.groupBy(keys2))
+//					.keyBy(keys1)
+//					.connect(op.input2.keyBy(keys2))
 //					.addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
 //							op.slideInterval, op.timeStamp1, op.timeStamp2));
 			return null;
@@ -244,8 +244,8 @@ public class StreamJoinOperator<I1, I2> extends
 
 //				return new JoinedStream<I1, I2, OUT>(
 //						predicate, predicate.op.input1
-//						.groupBy(predicate.keys1)
-//						.connect(predicate.op.input2.groupBy(predicate.keys2))
+//						.keyBy(predicate.keys1)
+//						.connect(predicate.op.input2.keyBy(predicate.keys2))
 //						.addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize,
 //								predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2));
 				return null;

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
index 7ea1309..0f9cbe9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
@@ -59,7 +59,7 @@ public class CoStreamTest extends StreamingMultipleProgramsTestBase {
 			public boolean filter(Integer value) throws Exception {
 				return true;
 			}
-		}).groupBy(new KeySelector<Integer, Integer>() {
+		}).keyBy(new KeySelector<Integer, Integer>() {
 
 			private static final long serialVersionUID = 1L;
 
@@ -88,7 +88,7 @@ public class CoStreamTest extends StreamingMultipleProgramsTestBase {
 					public boolean filter(Tuple2<Integer, Integer> value) throws Exception {
 						return true;
 					}
-				}).disableChaining().groupBy(new KeySelector<Tuple2<Integer, Integer>, Integer>() {
+				}).disableChaining().keyBy(new KeySelector<Tuple2<Integer, Integer>, Integer>() {
 
 					private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 337d97b..55bf889 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -38,7 +38,7 @@ import org.apache.flink.streaming.api.datastream.ConnectedStreams;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.GroupedDataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitDataStream;
 import org.apache.flink.streaming.api.datastream.WindowedDataStream;
@@ -125,7 +125,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 	}
 
 	/**
-	 * Tests that {@link DataStream#groupBy} and {@link DataStream#partitionByHash} result in
+	 * Tests that {@link DataStream#keyBy} and {@link DataStream#partitionByHash} result in
 	 * different and correct topologies. Does the some for the {@link ConnectedStreams}.
 	 */
 	@Test
@@ -138,10 +138,10 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		ConnectedStreams connected = src1.connect(src2);
 
 		//Testing DataStream grouping
-		DataStream group1 = src1.groupBy(0);
-		DataStream group2 = src1.groupBy(1, 0);
-		DataStream group3 = src1.groupBy("f0");
-		DataStream group4 = src1.groupBy(new FirstSelector());
+		DataStream group1 = src1.keyBy(0);
+		DataStream group2 = src1.keyBy(1, 0);
+		DataStream group3 = src1.keyBy("f0");
+		DataStream group4 = src1.keyBy(new FirstSelector());
 
 		int id1 = createDownStreamId(group1);
 		int id2 = createDownStreamId(group2);
@@ -153,10 +153,10 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id3)));
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id4)));
 
-		assertTrue(isGrouped(group1));
-		assertTrue(isGrouped(group2));
-		assertTrue(isGrouped(group3));
-		assertTrue(isGrouped(group4));
+		assertTrue(isKeyed(group1));
+		assertTrue(isKeyed(group2));
+		assertTrue(isKeyed(group3));
+		assertTrue(isKeyed(group4));
 
 		//Testing DataStream partitioning
 		DataStream partition1 = src1.partitionByHash(0);
@@ -174,10 +174,10 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid3)));
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid4)));
 
-		assertFalse(isGrouped(partition1));
-		assertFalse(isGrouped(partition3));
-		assertFalse(isGrouped(partition2));
-		assertFalse(isGrouped(partition4));
+		assertFalse(isKeyed(partition1));
+		assertFalse(isKeyed(partition3));
+		assertFalse(isKeyed(partition2));
+		assertFalse(isKeyed(partition4));
 
 		// Testing DataStream custom partitioning
 		Partitioner<Long> longPartitioner = new Partitioner<Long>() {
@@ -199,24 +199,24 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid2)));
 		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid3)));
 
-		assertFalse(isGrouped(customPartition1));
-		assertFalse(isGrouped(customPartition3));
-		assertFalse(isGrouped(customPartition4));
+		assertFalse(isKeyed(customPartition1));
+		assertFalse(isKeyed(customPartition3));
+		assertFalse(isKeyed(customPartition4));
 
 		//Testing ConnectedStreams grouping
-		ConnectedStreams connectedGroup1 = connected.groupBy(0, 0);
+		ConnectedStreams connectedGroup1 = connected.keyBy(0, 0);
 		Integer downStreamId1 = createDownStreamId(connectedGroup1);
 
-		ConnectedStreams connectedGroup2 = connected.groupBy(new int[]{0}, new int[]{0});
+		ConnectedStreams connectedGroup2 = connected.keyBy(new int[]{0}, new int[]{0});
 		Integer downStreamId2 = createDownStreamId(connectedGroup2);
 
-		ConnectedStreams connectedGroup3 = connected.groupBy("f0", "f0");
+		ConnectedStreams connectedGroup3 = connected.keyBy("f0", "f0");
 		Integer downStreamId3 = createDownStreamId(connectedGroup3);
 
-		ConnectedStreams connectedGroup4 = connected.groupBy(new String[]{"f0"}, new String[]{"f0"});
+		ConnectedStreams connectedGroup4 = connected.keyBy(new String[]{"f0"}, new String[]{"f0"});
 		Integer downStreamId4 = createDownStreamId(connectedGroup4);
 
-		ConnectedStreams connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector());
+		ConnectedStreams connectedGroup5 = connected.keyBy(new FirstSelector(), new FirstSelector());
 		Integer downStreamId5 = createDownStreamId(connectedGroup5);
 
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1)));
@@ -234,11 +234,11 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId5)));
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId5)));
 
-		assertTrue(isGrouped(connectedGroup1));
-		assertTrue(isGrouped(connectedGroup2));
-		assertTrue(isGrouped(connectedGroup3));
-		assertTrue(isGrouped(connectedGroup4));
-		assertTrue(isGrouped(connectedGroup5));
+		assertTrue(isKeyed(connectedGroup1));
+		assertTrue(isKeyed(connectedGroup2));
+		assertTrue(isKeyed(connectedGroup3));
+		assertTrue(isKeyed(connectedGroup4));
+		assertTrue(isKeyed(connectedGroup5));
 
 		//Testing ConnectedStreams partitioning
 		ConnectedStreams connectedPartition1 = connected.partitionByHash(0, 0);
@@ -281,11 +281,11 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
 				connectDownStreamId5)));
 
-		assertFalse(isGrouped(connectedPartition1));
-		assertFalse(isGrouped(connectedPartition2));
-		assertFalse(isGrouped(connectedPartition3));
-		assertFalse(isGrouped(connectedPartition4));
-		assertFalse(isGrouped(connectedPartition5));
+		assertFalse(isKeyed(connectedPartition1));
+		assertFalse(isKeyed(connectedPartition2));
+		assertFalse(isKeyed(connectedPartition3));
+		assertFalse(isKeyed(connectedPartition4));
+		assertFalse(isKeyed(connectedPartition5));
 	}
 
 	/**
@@ -601,8 +601,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return dataStream.print().getTransformation().getId();
 	}
 
-	private static boolean isGrouped(DataStream dataStream) {
-		return dataStream instanceof GroupedDataStream;
+	private static boolean isKeyed(DataStream dataStream) {
+		return dataStream instanceof KeyedStream;
 	}
 
 	private static Integer createDownStreamId(ConnectedStreams dataStream) {
@@ -621,8 +621,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return coMap.getId();
 	}
 
-	private static boolean isGrouped(ConnectedStreams dataStream) {
-		return (dataStream.getFirstInput() instanceof GroupedDataStream && dataStream.getSecondInput() instanceof GroupedDataStream);
+	private static boolean isKeyed(ConnectedStreams dataStream) {
+		return (dataStream.getFirstInput() instanceof KeyedStream && dataStream.getSecondInput() instanceof KeyedStream);
 	}
 
 	private static boolean isPartitioned(StreamEdge edge) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
index 774f58d..43371b7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
@@ -401,7 +401,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 				.withFeedbackType("String");
 
 		try {
-			coIt.groupBy(1, 2);
+			coIt.keyBy(1, 2);
 			fail();
 		} catch (InvalidProgramException e) {
 			// this is expected
@@ -476,7 +476,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source = env.fromElements(1, 2, 3)
 				.map(NoOpIntMap).name("ParallelizeMap");
 
-		IterativeDataStream<Integer> it = source.groupBy(key).iterate(3000);
+		IterativeDataStream<Integer> it = source.keyBy(key).iterate(3000);
 
 		DataStream<Integer> head = it.flatMap(new RichFlatMapFunction<Integer, Integer>() {
 
@@ -502,7 +502,7 @@ public class IterateTest extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		it.closeWith(head.groupBy(key).union(head.map(NoOpIntMap).groupBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
+		it.closeWith(head.keyBy(key).union(head.map(NoOpIntMap).keyBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
 
 		env.execute();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9baadfe8/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
index 11100a4..6401546 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
@@ -78,7 +78,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<Integer, Integer>> sourceStream = env.addSource(new TupleSource(numElements, numKeys));
 
 		SplitDataStream<Tuple2<Integer, Integer>> splittedResult = sourceStream
-			.groupBy(0)
+			.keyBy(0)
 			.fold(0, new FoldFunction<Tuple2<Integer, Integer>, Integer>() {
 				@Override
 				public Integer fold(Integer accumulator, Tuple2<Integer, Integer> value) throws Exception {
@@ -146,7 +146,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
 
 		input
-			.groupBy(0)
+			.keyBy(0)
 			.fold(
 				new NonSerializable(42),
 				new FoldFunction<Tuple2<Integer, NonSerializable>, NonSerializable>() {


[05/13] flink git commit: [FLINK-2550] Simplify Stream Java API Class Names

Posted by al...@apache.org.
[FLINK-2550] Simplify Stream Java API Class Names

KeyedDataStream -> KeyedStream
KeyedWindowDataStream -> WindowedStream
NonParallelWindowDataStream -> AllWindowedStream

KeyedWindowFunction -> WindowFunction
WindowFunction -> AllWindowFunction
(along with rich functions and reduce function wrappers)

WindowedStream.mapWindow -> WindowedStream.apply
AllWindowedStream.mapWindow -> AllWindowedStream.apply

Also renamed the tests to match the new names.


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

Branch: refs/heads/master
Commit: 9e6e0aeca01c50640827adbdd60089761cd5e8d2
Parents: 68c1afc
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Oct 1 15:58:52 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       | 231 ++++++++++++++
 .../streaming/api/datastream/DataStream.java    |  34 +--
 .../api/datastream/GroupedDataStream.java       |   2 +-
 .../api/datastream/KeyedDataStream.java         | 159 ----------
 .../streaming/api/datastream/KeyedStream.java   | 160 ++++++++++
 .../api/datastream/KeyedWindowDataStream.java   | 287 ------------------
 .../datastream/NonParallelWindowDataStream.java | 218 -------------
 .../api/datastream/WindowedStream.java          | 302 +++++++++++++++++++
 .../functions/windowing/AllWindowFunction.java  |  45 +++
 .../windowing/KeyedWindowFunction.java          |  45 ---
 .../windowing/ReduceAllWindowFunction.java      |  70 +++++
 .../windowing/ReduceKeyedWindowFunction.java    |  70 -----
 .../windowing/ReduceWindowFunction.java         |   4 +-
 .../ReduceWindowFunctionWithWindow.java         |   4 +-
 .../windowing/RichAllWindowFunction.java        |  25 ++
 .../windowing/RichKeyedWindowFunction.java      |  25 --
 .../functions/windowing/RichWindowFunction.java |   2 +-
 .../api/functions/windowing/WindowFunction.java |  16 +-
 .../windowing/AccumulatingKeyedTimePanes.java   |  14 +-
 ...ccumulatingProcessingTimeWindowOperator.java |   8 +-
 .../EvictingNonKeyedWindowOperator.java         |   6 +-
 .../windowing/EvictingWindowOperator.java       |   6 +-
 .../windowing/NonKeyedWindowOperator.java       |   8 +-
 .../operators/windowing/WindowOperator.java     |   8 +-
 .../runtime/tasks/StreamingRuntimeContext.java  |   2 +-
 .../api/state/StatefulOperatorTest.java         |   4 +-
 ...AlignedProcessingTimeWindowOperatorTest.java |  22 +-
 .../windowing/AllWindowTranslationTest.java     | 198 ++++++++++++
 .../EvictingNonKeyedWindowOperatorTest.java     |   5 +-
 .../windowing/EvictingWindowOperatorTest.java   |   4 +-
 .../windowing/NonKeyedWindowOperatorTest.java   |  11 +-
 ...ParallelWindowDataStreamTranslationTest.java | 198 ------------
 .../windowing/TimeWindowTranslationTest.java    |  13 +-
 .../operators/windowing/WindowOperatorTest.java |  10 +-
 .../windowing/WindowTranslationTest.java        |  17 +-
 .../GroupedProcessingTimeWindowExample.java     |  10 +-
 .../flink/streaming/api/scala/DataStream.scala  |   5 +-
 37 files changed, 1144 insertions(+), 1104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
new file mode 100644
index 0000000..e5c7c18
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -0,0 +1,231 @@
+/*
+ * 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.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.evictors.Evictor;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.EvictingNonKeyedWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.NonKeyedWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+
+/**
+ * A {@code AllWindowedStream} represents a data stream where the stream of
+ * elements is split into windows based on a
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
+ * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ *
+ * <p>
+ * If an {@link org.apache.flink.streaming.api.windowing.evictors.Evictor} is specified it will be
+ * used to evict elements from the window after
+ * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
+ * When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ *
+ * <p>
+ * Note that the {@code AllWindowedStream} is purely and API construct, during runtime
+ * the {@code AllWindowedStream} will be collapsed together with the
+ * operation over the window into one single operation.
+ *
+ * @param <T> The type of elements in the stream.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
+ */
+public class AllWindowedStream<T, W extends Window> {
+
+	/** The data stream that is windowed by this stream */
+	private final DataStream<T> input;
+
+	/** The window assigner */
+	private final WindowAssigner<? super T, W> windowAssigner;
+
+	/** The trigger that is used for window evaluation/emission. */
+	private Trigger<? super T, ? super W> trigger;
+
+	/** The evictor that is used for evicting elements before window evaluation. */
+	private Evictor<? super T, ? super W> evictor;
+
+
+	public AllWindowedStream(DataStream<T> input,
+			WindowAssigner<? super T, W> windowAssigner) {
+		this.input = input;
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger();
+	}
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public AllWindowedStream<T, W> trigger(Trigger<? super T, ? super W> trigger) {
+		this.trigger = trigger;
+		return this;
+	}
+
+	/**
+	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+	 *
+	 * <p>
+	 * Note: When using an evictor window performance will degrade significantly, since
+	 * pre-aggregation of window results cannot be used.
+	 */
+	public AllWindowedStream<T, W> evictor(Evictor<? super T, ? super W> evictor) {
+		this.evictor = evictor;
+		return this;
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Operations on the keyed windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies a reduce function to the window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the reduce function is interpreted
+	 * as a regular non-windowed stream.
+	 * <p>
+	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
+	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
+	 * so a few elements are stored per key (one per slide interval).
+	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+	 * aggregation tree.
+	 * 
+	 * @param function The reduce function.
+	 * @return The data stream that is the result of applying the reduce function to the window. 
+	 */
+	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "Reduce at " + callLocation;
+
+		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		if (result != null) {
+			return result;
+		}
+
+		String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, T> operator;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceAllWindowFunction<W, T>(function),
+					trigger,
+					evictor);
+
+		} else {
+			// we need to copy because we need our own instance of the pre aggregator
+			@SuppressWarnings("unchecked")
+			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
+
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceAllWindowFunction<W, T>(function),
+					trigger);
+		}
+
+		return input.transform(opName, input.getType(), operator).setParallelism(1);
+	}
+
+	/**
+	 * Applies a window function to the window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the window function is interpreted
+	 * as a regular non-windowed stream.
+	 * <p>
+	 * Not that this function requires that all data in the windows is buffered until the window
+	 * is evaluated, as the function provides no means of pre-aggregation.
+	 * 
+	 * @param function The window function.
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> DataStream<R> apply(AllWindowFunction<T, R, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, AllWindowFunction.class, true, true, inType, null, false);
+
+		return apply(function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the window function is interpreted
+	 * as a regular non-windowed stream.
+	 * <p>
+	 * Not that this function requires that all data in the windows is buffered until the window
+	 * is evaluated, as the function provides no means of pre-aggregation.
+	 *
+	 * @param function The window function.
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> DataStream<R> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "MapWindow at " + callLocation;
+
+		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, R> operator;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor);
+
+		} else {
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger);
+		}
+
+		return input.transform(opName, resultType, operator).setParallelism(1);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+
+	private <R> DataStream<R> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		// TODO: add once non-parallel fast aligned time windows operator is ready
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index c2be055..ad159f9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -233,15 +233,15 @@ public class DataStream<T> {
 
 	/**
 	 * 
-	 * It creates a new {@link KeyedDataStream} that uses the provided key for partitioning
+	 * It creates a new {@link KeyedStream} that uses the provided key for partitioning
 	 * its operator states. 
 	 *
 	 * @param key
 	 *            The KeySelector to be used for extracting the key for partitioning
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
 	 */
-	public <K> KeyedDataStream<T, K> keyBy(KeySelector<T, K> key){
-		return new KeyedDataStream<T, K>(this, clean(key));
+	public <K> KeyedStream<T, K> keyBy(KeySelector<T, K> key){
+		return new KeyedStream<T, K>(this, clean(key));
 	}
 
 	/**
@@ -250,9 +250,9 @@ public class DataStream<T> {
 	 * @param fields
 	 *            The position of the fields on which the {@link DataStream}
 	 *            will be grouped.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
 	 */
-	public KeyedDataStream<T, Tuple> keyBy(int... fields) {
+	public KeyedStream<T, Tuple> keyBy(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
 			return keyBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
 		} else {
@@ -269,14 +269,14 @@ public class DataStream<T> {
 	 * @param fields
 	 *            One or more field expressions on which the state of the {@link DataStream} operators will be
 	 *            partitioned.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
 	 **/
-	public KeyedDataStream<T, Tuple> keyBy(String... fields) {
+	public KeyedStream<T, Tuple> keyBy(String... fields) {
 		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
 	}
 
-	private KeyedDataStream<T, Tuple> keyBy(Keys<T> keys) {
-		return new KeyedDataStream<T, Tuple>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
+	private KeyedStream<T, Tuple> keyBy(Keys<T> keys) {
+		return new KeyedStream<T, Tuple>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
 				getType(), getExecutionConfig())));
 	}
 	
@@ -288,7 +288,7 @@ public class DataStream<T> {
 	 * @param fields
 	 *            The position of the fields on which the states of the {@link DataStream}
 	 *            will be partitioned.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedDataStream)
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
 	 */
 	public GroupedDataStream<T, Tuple> groupBy(int... fields) {
 		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
@@ -816,7 +816,7 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Windows this {@code KeyedDataStream} into tumbling time windows.
+	 * Windows this {@code DataStream} into tumbling time windows.
 	 *
 	 * <p>
 	 * This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
@@ -826,7 +826,7 @@ public class DataStream<T> {
 	 *
 	 * @param size The size of the window.
 	 */
-	public NonParallelWindowDataStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
+	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
 		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
 		if (actualSize instanceof EventTime) {
@@ -837,7 +837,7 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Windows this {@code KeyedDataStream} into sliding time windows.
+	 * Windows this {@code DataStream} into sliding time windows.
 	 *
 	 * <p>
 	 * This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
@@ -847,7 +847,7 @@ public class DataStream<T> {
 	 *
 	 * @param size The size of the window.
 	 */
-	public NonParallelWindowDataStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
+	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
 		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
 
@@ -879,8 +879,8 @@ public class DataStream<T> {
 	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
 	 * @return The trigger windows data stream.
 	 */
-	public <W extends Window> NonParallelWindowDataStream<T, W> windowAll(WindowAssigner<? super T, W> assigner) {
-		return new NonParallelWindowDataStream<>(this, assigner);
+	public <W extends Window> AllWindowedStream<T, W> windowAll(WindowAssigner<? super T, W> assigner) {
+		return new AllWindowedStream<>(this, assigner);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
index fde5a6d..ebaeb56 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/GroupedDataStream.java
@@ -39,7 +39,7 @@ import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
  * @param <T> The type of the elements in the Grouped Stream.
  * @param <KEY> The type of the key in the Keyed Stream.
  */
-public class GroupedDataStream<T, KEY> extends KeyedDataStream<T, KEY> {
+public class GroupedDataStream<T, KEY> extends KeyedStream<T, KEY> {
 
 	/**
 	 * Creates a new {@link GroupedDataStream}, group inclusion is determined using

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
deleted file mode 100644
index 2ae07b2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
+++ /dev/null
@@ -1,159 +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 org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.transformations.PartitionTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.time.EventTime;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-
-/**
- * A KeyedDataStream represents a {@link DataStream} on which operator state is
- * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a {@link DataStream}
- * are also possible on a KeyedDataStream, with the exception of partitioning methods such as shuffle, forward and groupBy.
- * 
- * 
- * @param <T> The type of the elements in the Keyed Stream.
- * @param <KEY> The type of the key in the Keyed Stream.
- */
-public class KeyedDataStream<T, KEY> extends DataStream<T> {
-	
-	protected final KeySelector<T, KEY> keySelector;
-
-	/**
-	 * Creates a new {@link KeyedDataStream} using the given {@link KeySelector}
-	 * to partition operator state by key.
-	 * 
-	 * @param dataStream
-	 *            Base stream of data
-	 * @param keySelector
-	 *            Function for determining state partitions
-	 */
-	public KeyedDataStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
-		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<T>(dataStream.getTransformation(), new HashPartitioner<T>(keySelector)));
-		this.keySelector = keySelector;
-	}
-
-	
-	public KeySelector<T, KEY> getKeySelector() {
-		return this.keySelector;
-	}
-
-	
-	@Override
-	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
-		throw new UnsupportedOperationException("Cannot override partitioning for KeyedDataStream.");
-	}
-
-	
-	@Override
-	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
-			TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
-
-		SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
-
-		((OneInputTransformation<T, R>) returnStream.getTransformation()).setStateKeySelector(keySelector);
-		return returnStream;
-	}
-
-	
-	
-	@Override
-	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
-		DataStreamSink<T> result = super.addSink(sinkFunction);
-		result.getTransformation().setStateKeySelector(keySelector);
-		return result;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Windowing
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Windows this {@code KeyedDataStream} into tumbling time windows.
-	 *
-	 * <p>
-	 * This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
-	 * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
-	 * set using
-	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
-	 *
-	 * @param size The size of the window.
-	 */
-	public KeyedWindowDataStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
-		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
-		if (actualSize instanceof EventTime) {
-			return window(TumblingTimeWindows.of(actualSize.toMilliseconds()));
-		} else {
-			return window(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
-		}
-	}
-
-	/**
-	 * Windows this {@code KeyedDataStream} into sliding time windows.
-	 *
-	 * <p>
-	 * This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
-	 * {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
-	 * set using
-	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
-	 *
-	 * @param size The size of the window.
-	 */
-	public KeyedWindowDataStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
-		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
-		if (actualSize instanceof EventTime) {
-			return window(SlidingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
-		} else {
-			return window(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
-		}
-	}
-
-	/**
-	 * Windows this data stream to a {@code KeyedWindowDataStream}, which evaluates windows
-	 * over a key grouped stream. Elements are put into windows by a {@link WindowAssigner}. The
-	 * grouping of elements is done both by key and by window.
-	 *
-	 * <p>
-	 * A {@link org.apache.flink.streaming.api.windowing.triggers.Trigger} can be defined to specify
-	 * when windows are evaluated. However, {@code WindowAssigners} have a default {@code Trigger}
-	 * that is used if a {@code Trigger} is not specified.
-	 *
-	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
-	 * @return The trigger windows data stream.
-	 */
-	public <W extends Window> KeyedWindowDataStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
-		return new KeyedWindowDataStream<>(this, assigner);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
new file mode 100644
index 0000000..b3cfb55
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -0,0 +1,160 @@
+/*
+ * 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.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+import org.apache.flink.streaming.api.windowing.time.EventTime;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+
+/**
+ * A {@code KeyedStream} represents a {@link DataStream} on which operator state is
+ * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a
+ * {@code DataStream} are also possible on a {@code KeyedStream}, with the exception of
+ * partitioning methods such as shuffle, forward and groupBy.
+ * 
+ * 
+ * @param <T> The type of the elements in the Keyed Stream.
+ * @param <KEY> The type of the key in the Keyed Stream.
+ */
+public class KeyedStream<T, KEY> extends DataStream<T> {
+	
+	protected final KeySelector<T, KEY> keySelector;
+
+	/**
+	 * Creates a new {@link KeyedStream} using the given {@link KeySelector}
+	 * to partition operator state by key.
+	 * 
+	 * @param dataStream
+	 *            Base stream of data
+	 * @param keySelector
+	 *            Function for determining state partitions
+	 */
+	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
+		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<T>(dataStream.getTransformation(), new HashPartitioner<T>(keySelector)));
+		this.keySelector = keySelector;
+	}
+
+	
+	public KeySelector<T, KEY> getKeySelector() {
+		return this.keySelector;
+	}
+
+	
+	@Override
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		throw new UnsupportedOperationException("Cannot override partitioning for KeyedStream.");
+	}
+
+	
+	@Override
+	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
+			TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
+
+		SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
+
+		((OneInputTransformation<T, R>) returnStream.getTransformation()).setStateKeySelector(keySelector);
+		return returnStream;
+	}
+
+	
+	
+	@Override
+	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
+		DataStreamSink<T> result = super.addSink(sinkFunction);
+		result.getTransformation().setStateKeySelector(keySelector);
+		return result;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Windowing
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Windows this {@code KeyedStream} into tumbling time windows.
+	 *
+	 * <p>
+	 * This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
+	 * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
+	 */
+	public WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return window(TumblingTimeWindows.of(actualSize.toMilliseconds()));
+		} else {
+			return window(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
+		}
+	}
+
+	/**
+	 * Windows this {@code KeyedStream} into sliding time windows.
+	 *
+	 * <p>
+	 * This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
+	 * {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
+	 */
+	public WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return window(SlidingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+		} else {
+			return window(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+		}
+	}
+
+	/**
+	 * Windows this data stream to a {@code WindowedStream}, which evaluates windows
+	 * over a key grouped stream. Elements are put into windows by a {@link WindowAssigner}. The
+	 * grouping of elements is done both by key and by window.
+	 *
+	 * <p>
+	 * A {@link org.apache.flink.streaming.api.windowing.triggers.Trigger} can be defined to specify
+	 * when windows are evaluated. However, {@code WindowAssigners} have a default {@code Trigger}
+	 * that is used if a {@code Trigger} is not specified.
+	 *
+	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
+	 * @return The trigger windows data stream.
+	 */
+	public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
+		return new WindowedStream<>(this, assigner);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
deleted file mode 100644
index 9d05b8c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
+++ /dev/null
@@ -1,287 +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 org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.evictors.Evictor;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-
-/**
- * A {@code KeyedWindowDataStream} represents a data stream where elements are grouped by
- * key, and for each key, the stream of elements is split into windows based on a
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
- * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
- *
- * <p>
- * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
- * different points for each key.
- *
- * <p>
- * If an {@link Evictor} is specified it will be used to evict elements from the window after
- * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
- * When using an evictor window performance will degrade significantly, since
- * pre-aggregation of window results cannot be used.
- *
- * <p>
- * Note that the {@code KeyedWindowDataStream} is purely and API construct, during runtime
- * the {@code KeyedWindowDataStream} will be collapsed together with the
- * {@code KeyedDataStream} and the operation over the window into one single operation.
- * 
- * @param <T> The type of elements in the stream.
- * @param <K> The type of the key by which elements are grouped.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
- */
-public class KeyedWindowDataStream<T, K, W extends Window> {
-
-	/** The keyed data stream that is windowed by this stream */
-	private final KeyedDataStream<T, K> input;
-
-	/** The window assigner */
-	private final WindowAssigner<? super T, W> windowAssigner;
-
-	/** The trigger that is used for window evaluation/emission. */
-	private Trigger<? super T, ? super W> trigger;
-
-	/** The evictor that is used for evicting elements before window evaluation. */
-	private Evictor<? super T, ? super W> evictor;
-
-
-	public KeyedWindowDataStream(KeyedDataStream<T, K> input,
-			WindowAssigner<? super T, W> windowAssigner) {
-		this.input = input;
-		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger();
-	}
-
-	/**
-	 * Sets the {@code Trigger} that should be used to trigger window emission.
-	 */
-	public KeyedWindowDataStream<T, K, W> trigger(Trigger<? super T, ? super W> trigger) {
-		this.trigger = trigger;
-		return this;
-	}
-
-	/**
-	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
-	 *
-	 * <p>
-	 * Note: When using an evictor window performance will degrade significantly, since
-	 * pre-aggregation of window results cannot be used.
-	 */
-	public KeyedWindowDataStream<T, K, W> evictor(Evictor<? super T, ? super W> evictor) {
-		this.evictor = evictor;
-		return this;
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Operations on the keyed windows
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies a reduce function to the window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the reduce function is interpreted
-	 * as a regular non-windowed stream.
-	 * <p>
-	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
-	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
-	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
-	 * so a few elements are stored per key (one per slide interval).
-	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
-	 * aggregation tree.
-	 * 
-	 * @param function The reduce function.
-	 * @return The data stream that is the result of applying the reduce function to the window. 
-	 */
-	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "Reduce at " + callLocation;
-
-		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
-		if (result != null) {
-			return result;
-		}
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, T> operator;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					new ReduceKeyedWindowFunction<K, W, T>(function),
-					trigger,
-					evictor);
-
-		} else {
-			// we need to copy because we need our own instance of the pre aggregator
-			@SuppressWarnings("unchecked")
-			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
-
-			operator = new WindowOperator<>(windowAssigner,
-					keySel,
-					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
-					new ReduceKeyedWindowFunction<K, W, T>(function),
-					trigger);
-		}
-
-		return input.transform(opName, input.getType(), operator);
-	}
-
-	/**
-	 * Applies a window function to the window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the window function is interpreted
-	 * as a regular non-windowed stream.
-	 * <p>
-	 * Not that this function requires that all data in the windows is buffered until the window
-	 * is evaluated, as the function provides no means of pre-aggregation.
-	 * 
-	 * @param function The window function.
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	public <R> DataStream<R> mapWindow(KeyedWindowFunction<T, R, K, W> function) {
-		// clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-		
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, KeyedWindowFunction.class, true, true, inType, null, false);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "MapWindow at " + callLocation;
-
-		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, R> operator;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor);
-
-		} else {
-			operator = new WindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger);
-		}
-
-
-
-		return input.transform(opName, resultType, operator);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private <R> DataStream<R> createFastTimeOperatorIfValid(
-			Function function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSlide();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			if (function instanceof ReduceFunction) {
-				@SuppressWarnings("unchecked")
-				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
-
-				@SuppressWarnings("unchecked")
-				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
-						new AggregatingProcessingTimeWindowOperator<>(
-								reducer, input.getKeySelector(), windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-			else if (function instanceof KeyedWindowFunction) {
-				@SuppressWarnings("unchecked")
-				KeyedWindowFunction<T, R, K, TimeWindow> wf = (KeyedWindowFunction<T, R, K, TimeWindow>) function;
-
-				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-						wf, input.getKeySelector(), windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-		} else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSize();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			if (function instanceof ReduceFunction) {
-				@SuppressWarnings("unchecked")
-				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
-
-				@SuppressWarnings("unchecked")
-				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
-						new AggregatingProcessingTimeWindowOperator<>(
-								reducer, input.getKeySelector(), windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-			else if (function instanceof KeyedWindowFunction) {
-				@SuppressWarnings("unchecked")
-				KeyedWindowFunction<T, R, K, TimeWindow> wf = (KeyedWindowFunction<T, R, K, TimeWindow>) function;
-
-				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-						wf, input.getKeySelector(), windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-		}
-
-		return null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
deleted file mode 100644
index 5cb3b6b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
+++ /dev/null
@@ -1,218 +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 org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.evictors.Evictor;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.EvictingNonKeyedWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.NonKeyedWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-
-/**
- * A {@code NonParallelWindowDataStream} represents a data stream where the stream of
- * elements is split into windows based on a
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
- * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
- *
- * <p>
- * If an {@link org.apache.flink.streaming.api.windowing.evictors.Evictor} is specified it will be
- * used to evict elements from the window after
- * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
- * When using an evictor window performance will degrade significantly, since
- * pre-aggregation of window results cannot be used.
- *
- * <p>
- * Note that the {@code NonParallelWindowDataStream} is purely and API construct, during runtime
- * the {@code NonParallelWindowDataStream} will be collapsed together with the
- * operation over the window into one single operation.
- *
- * @param <T> The type of elements in the stream.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
- */
-public class NonParallelWindowDataStream<T, W extends Window> {
-
-	/** The data stream that is windowed by this stream */
-	private final DataStream<T> input;
-
-	/** The window assigner */
-	private final WindowAssigner<? super T, W> windowAssigner;
-
-	/** The trigger that is used for window evaluation/emission. */
-	private Trigger<? super T, ? super W> trigger;
-
-	/** The evictor that is used for evicting elements before window evaluation. */
-	private Evictor<? super T, ? super W> evictor;
-
-
-	public NonParallelWindowDataStream(DataStream<T> input,
-			WindowAssigner<? super T, W> windowAssigner) {
-		this.input = input;
-		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger();
-	}
-
-	/**
-	 * Sets the {@code Trigger} that should be used to trigger window emission.
-	 */
-	public NonParallelWindowDataStream<T, W> trigger(Trigger<? super T, ? super W> trigger) {
-		this.trigger = trigger;
-		return this;
-	}
-
-	/**
-	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
-	 *
-	 * <p>
-	 * Note: When using an evictor window performance will degrade significantly, since
-	 * pre-aggregation of window results cannot be used.
-	 */
-	public NonParallelWindowDataStream<T, W> evictor(Evictor<? super T, ? super W> evictor) {
-		this.evictor = evictor;
-		return this;
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Operations on the keyed windows
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies a reduce function to the window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the reduce function is interpreted
-	 * as a regular non-windowed stream.
-	 * <p>
-	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
-	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
-	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
-	 * so a few elements are stored per key (one per slide interval).
-	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
-	 * aggregation tree.
-	 * 
-	 * @param function The reduce function.
-	 * @return The data stream that is the result of applying the reduce function to the window. 
-	 */
-	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "Reduce at " + callLocation;
-
-		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
-		if (result != null) {
-			return result;
-		}
-
-		String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-
-		OneInputStreamOperator<T, T> operator;
-
-		if (evictor != null) {
-			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
-					new HeapWindowBuffer.Factory<T>(),
-					new ReduceWindowFunction<W, T>(function),
-					trigger,
-					evictor);
-
-		} else {
-			// we need to copy because we need our own instance of the pre aggregator
-			@SuppressWarnings("unchecked")
-			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
-
-			operator = new NonKeyedWindowOperator<>(windowAssigner,
-					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
-					new ReduceWindowFunction<W, T>(function),
-					trigger);
-		}
-
-		return input.transform(opName, input.getType(), operator).setParallelism(1);
-	}
-
-	/**
-	 * Applies a window function to the window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the window function is interpreted
-	 * as a regular non-windowed stream.
-	 * <p>
-	 * Not that this function requires that all data in the windows is buffered until the window
-	 * is evaluated, as the function provides no means of pre-aggregation.
-	 * 
-	 * @param function The window function.
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	public <R> DataStream<R> mapWindow(WindowFunction<T, R, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, WindowFunction.class, true, true, inType, null, false);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "MapWindow at " + callLocation;
-
-		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-
-		OneInputStreamOperator<T, R> operator;
-
-		if (evictor != null) {
-			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor);
-
-		} else {
-			operator = new NonKeyedWindowOperator<>(windowAssigner,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger);
-		}
-
-
-
-		return input.transform(opName, resultType, operator).setParallelism(1);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-
-	private <R> DataStream<R> createFastTimeOperatorIfValid(
-			Function function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		// TODO: add once non-parallel fast aligned time windows operator is ready
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
new file mode 100644
index 0000000..16898dd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -0,0 +1,302 @@
+/*
+ * 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.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.evictors.Evictor;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+
+/**
+ * A {@code WindowedStream} represents a data stream where elements are grouped by
+ * key, and for each key, the stream of elements is split into windows based on a
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
+ * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ *
+ * <p>
+ * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
+ * different points for each key.
+ *
+ * <p>
+ * If an {@link Evictor} is specified it will be used to evict elements from the window after
+ * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
+ * When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ *
+ * <p>
+ * Note that the {@code WindowedStream} is purely and API construct, during runtime
+ * the {@code WindowedStream} will be collapsed together with the
+ * {@code KeyedStream} and the operation over the window into one single operation.
+ * 
+ * @param <T> The type of elements in the stream.
+ * @param <K> The type of the key by which elements are grouped.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
+ */
+public class WindowedStream<T, K, W extends Window> {
+
+	/** The keyed data stream that is windowed by this stream */
+	private final KeyedStream<T, K> input;
+
+	/** The window assigner */
+	private final WindowAssigner<? super T, W> windowAssigner;
+
+	/** The trigger that is used for window evaluation/emission. */
+	private Trigger<? super T, ? super W> trigger;
+
+	/** The evictor that is used for evicting elements before window evaluation. */
+	private Evictor<? super T, ? super W> evictor;
+
+
+	public WindowedStream(KeyedStream<T, K> input,
+			WindowAssigner<? super T, W> windowAssigner) {
+		this.input = input;
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger();
+	}
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public WindowedStream<T, K, W> trigger(Trigger<? super T, ? super W> trigger) {
+		this.trigger = trigger;
+		return this;
+	}
+
+	/**
+	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+	 *
+	 * <p>
+	 * Note: When using an evictor window performance will degrade significantly, since
+	 * pre-aggregation of window results cannot be used.
+	 */
+	public WindowedStream<T, K, W> evictor(Evictor<? super T, ? super W> evictor) {
+		this.evictor = evictor;
+		return this;
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Operations on the keyed windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies a reduce function to the window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the reduce function is interpreted
+	 * as a regular non-windowed stream.
+	 * <p>
+	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
+	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
+	 * so a few elements are stored per key (one per slide interval).
+	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+	 * aggregation tree.
+	 * 
+	 * @param function The reduce function.
+	 * @return The data stream that is the result of applying the reduce function to the window. 
+	 */
+	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "Reduce at " + callLocation;
+
+		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		if (result != null) {
+			return result;
+		}
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, T> operator;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceWindowFunction<K, W, T>(function),
+					trigger,
+					evictor);
+
+		} else {
+			// we need to copy because we need our own instance of the pre aggregator
+			@SuppressWarnings("unchecked")
+			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
+
+			operator = new WindowOperator<>(windowAssigner,
+					keySel,
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceWindowFunction<K, W, T>(function),
+					trigger);
+		}
+
+		return input.transform(opName, input.getType(), operator);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Not that this function requires that all data in the windows is buffered until the window
+	 * is evaluated, as the function provides no means of pre-aggregation.
+	 * 
+	 * @param function The window function.
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> DataStream<R> apply(WindowFunction<T, R, K, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, WindowFunction.class, true, true, inType, null, false);
+
+		return apply(function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Not that this function requires that all data in the windows is buffered until the window
+	 * is evaluated, as the function provides no means of pre-aggregation.
+	 *
+	 * @param function The window function.
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> DataStream<R> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "MapWindow at " + callLocation;
+
+		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, R> operator;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor);
+
+		} else {
+			operator = new WindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger);
+		}
+
+		return input.transform(opName, resultType, operator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private <R> DataStream<R> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
+			final long windowLength = timeWindows.getSize();
+			final long windowSlide = timeWindows.getSlide();
+
+			String opName = "Fast " + timeWindows + " of " + functionName;
+
+			if (function instanceof ReduceFunction) {
+				@SuppressWarnings("unchecked")
+				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
+
+				@SuppressWarnings("unchecked")
+				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
+						new AggregatingProcessingTimeWindowOperator<>(
+								reducer, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof WindowFunction) {
+				@SuppressWarnings("unchecked")
+				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		} else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
+			final long windowLength = timeWindows.getSize();
+			final long windowSlide = timeWindows.getSize();
+
+			String opName = "Fast " + timeWindows + " of " + functionName;
+
+			if (function instanceof ReduceFunction) {
+				@SuppressWarnings("unchecked")
+				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
+
+				@SuppressWarnings("unchecked")
+				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
+						new AggregatingProcessingTimeWindowOperator<>(
+								reducer, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof WindowFunction) {
+				@SuppressWarnings("unchecked")
+				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		}
+
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java
new file mode 100644
index 0000000..1d54436
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.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.functions.windowing;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+
+/**
+ * Base interface for functions that are evaluated over non-keyed windows.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ */
+public interface AllWindowFunction<IN, OUT,  W extends Window> extends Function, Serializable {
+
+	/**
+	 * Evaluates the window and outputs none or several elements.
+	 *
+	 * @param window The window that is being evaluated.
+	 * @param values The elements in the window being evaluated.
+	 * @param out A collector for emitting elements.
+	 *
+	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+	 */
+	void apply(W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/KeyedWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/KeyedWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/KeyedWindowFunction.java
deleted file mode 100644
index 77ce53e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/KeyedWindowFunction.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.functions.windowing;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-
-/**
- * Base interface for functions that are evaluated over keyed (grouped) windows.
- *
- * @param <IN> The type of the input value.
- * @param <OUT> The type of the output value.
- * @param <KEY> The type of the key.
- */
-public interface KeyedWindowFunction<IN, OUT, KEY, W extends Window> extends Function, Serializable {
-
-	/**
-	 * 
-	 * @param key
-	 * @param values
-	 * @param out
-	 * 
-	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery. 
-	 */
-	void evaluate(KEY key, W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java
new file mode 100644
index 0000000..24855a5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.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.functions.windowing;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+public class ReduceAllWindowFunction<W extends Window, T> extends RichAllWindowFunction<T, T, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final ReduceFunction<T> reduceFunction;
+
+	public ReduceAllWindowFunction(ReduceFunction<T> reduceFunction) {
+		this.reduceFunction = reduceFunction;
+	}
+
+	@Override
+	public void setRuntimeContext(RuntimeContext ctx) {
+		super.setRuntimeContext(ctx);
+		FunctionUtils.setFunctionRuntimeContext(reduceFunction, ctx);
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		FunctionUtils.openFunction(reduceFunction, parameters);
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		FunctionUtils.closeFunction(reduceFunction);
+	}
+
+	@Override
+	public void apply(W window, Iterable<T> values, Collector<T> out) throws Exception {
+		T result = null;
+
+		for (T v: values) {
+			if (result == null) {
+				result = v;
+			} else {
+				result = reduceFunction.reduce(result, v);
+			}
+		}
+
+		if (result != null) {
+			out.collect(result);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java
deleted file mode 100644
index 70627f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.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.functions.windowing;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-public class ReduceKeyedWindowFunction<K, W extends Window, T> extends RichKeyedWindowFunction<T, T, K, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final ReduceFunction<T> reduceFunction;
-
-	public ReduceKeyedWindowFunction(ReduceFunction<T> reduceFunction) {
-		this.reduceFunction = reduceFunction;
-	}
-
-	@Override
-	public void setRuntimeContext(RuntimeContext ctx) {
-		super.setRuntimeContext(ctx);
-		FunctionUtils.setFunctionRuntimeContext(reduceFunction, ctx);
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		FunctionUtils.openFunction(reduceFunction, parameters);
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		FunctionUtils.closeFunction(reduceFunction);
-	}
-
-	@Override
-	public void evaluate(K k, W window, Iterable<T> values, Collector<T> out) throws Exception {
-		T result = null;
-
-		for (T v: values) {
-			if (result == null) {
-				result = v;
-			} else {
-				result = reduceFunction.reduce(result, v);
-			}
-		}
-
-		if (result != null) {
-			out.collect(result);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
index ba26218..042fe18 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
@@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
-public class ReduceWindowFunction<W extends Window, T> extends RichWindowFunction<T, T, W> {
+public class ReduceWindowFunction<K, W extends Window, T> extends RichWindowFunction<T, T, K, W> {
 	private static final long serialVersionUID = 1L;
 
 	private final ReduceFunction<T> reduceFunction;
@@ -52,7 +52,7 @@ public class ReduceWindowFunction<W extends Window, T> extends RichWindowFunctio
 	}
 
 	@Override
-	public void evaluate(W window, Iterable<T> values, Collector<T> out) throws Exception {
+	public void apply(K k, W window, Iterable<T> values, Collector<T> out) throws Exception {
 		T result = null;
 
 		for (T v: values) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9e6e0aec/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
index bceff82..6a472b1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
@@ -25,7 +25,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
-public class ReduceWindowFunctionWithWindow<K, W extends Window, T> extends RichKeyedWindowFunction<T, Tuple2<W, T>, K, W> {
+public class ReduceWindowFunctionWithWindow<K, W extends Window, T> extends RichWindowFunction<T, Tuple2<W, T>, K, W> {
 	private static final long serialVersionUID = 1L;
 
 	private final ReduceFunction<T> reduceFunction;
@@ -53,7 +53,7 @@ public class ReduceWindowFunctionWithWindow<K, W extends Window, T> extends Rich
 	}
 
 	@Override
-	public void evaluate(K k, W window, Iterable<T> values, Collector<Tuple2<W, T>> out) throws Exception {
+	public void apply(K k, W window, Iterable<T> values, Collector<Tuple2<W, T>> out) throws Exception {
 		T result = null;
 
 		for (T v: values) {


[07/13] flink git commit: [FLINK-2550] Rename ConnectedDataStream to ConnectedStreams, Remove some operations

Posted by al...@apache.org.
[FLINK-2550] Rename ConnectedDataStream to ConnectedStreams, Remove some operations

The removed operations are tricky and some of them are not working
correctly. For now, co-reduce, stream-cross and stream-join are
removed.

I'm planning to add a new join implementation based on tagged union
that uses the new windowing code.


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

Branch: refs/heads/master
Commit: 23d8e26438370d8c99c24c8b43d543e953775fd2
Parents: 9e6e0ae
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Oct 1 17:07:11 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 docs/apis/streaming_guide.md                    |  22 +-
 .../connectors/flume/FlumeTopology.java         |   2 +-
 .../api/datastream/ConnectedDataStream.java     | 502 ------------------
 .../api/datastream/ConnectedStreams.java        | 331 ++++++++++++
 .../streaming/api/datastream/DataStream.java    |  39 +-
 .../api/datastream/IterativeDataStream.java     |  54 +-
 .../temporal/StreamCrossOperator.java           | 115 -----
 .../datastream/temporal/StreamJoinOperator.java |  54 +-
 .../api/functions/co/CoReduceFunction.java      | 107 ----
 .../api/functions/co/CoWindowFunction.java      |  30 --
 .../api/functions/co/CrossWindowFunction.java   |  44 --
 .../api/functions/co/JoinWindowFunction.java    |  77 ---
 .../api/functions/co/RichCoReduceFunction.java  |  40 --
 .../api/functions/co/RichCoWindowFunction.java  |  34 --
 .../api/operators/co/CoStreamGroupedReduce.java |  77 ---
 .../api/operators/co/CoStreamReduce.java        |  86 ----
 .../api/operators/co/CoStreamWindow.java        | 228 ---------
 .../flink/streaming/api/DataStreamTest.java     |  39 +-
 .../apache/flink/streaming/api/IterateTest.java |  14 +-
 .../flink/streaming/api/TypeFillTest.java       |  46 --
 .../streaming/api/WindowCrossJoinTest.java      |  17 -
 .../api/graph/StreamGraphGeneratorTest.java     |   4 +-
 .../api/scala/ConnectedDataStream.scala         | 510 -------------------
 .../streaming/api/scala/ConnectedStreams.scala  | 353 +++++++++++++
 .../flink/streaming/api/scala/DataStream.scala  |  23 +-
 .../api/scala/StreamCrossOperator.scala         | 101 ----
 .../api/scala/StreamJoinOperator.scala          |  30 +-
 .../flink/streaming/api/scala/package.scala     |   4 +-
 .../streaming/api/scala/DataStreamTest.scala    |  32 +-
 .../StreamingScalaAPICompletenessTest.scala     |  21 +-
 30 files changed, 831 insertions(+), 2205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/docs/apis/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md
index 9c584ef..c437114 100644
--- a/docs/apis/streaming_guide.md
+++ b/docs/apis/streaming_guide.md
@@ -991,9 +991,9 @@ dataStream1 cross dataStream2 onWindow (windowing_params)
 ### Co operators
 
 Co operators allow the users to jointly transform two `DataStream`s of different types, providing a simple way to jointly manipulate streams with a shared state. It is designed to support joint stream transformations where union is not appropriate due to different data types, or in case the user needs explicit tracking of the origin of individual elements.
-Co operators can be applied to `ConnectedDataStream`s which represent two `DataStream`s of possibly different types. A `ConnectedDataStream` can be created by calling the `connect(otherDataStream)` method of a `DataStream`.
+Co operators can be applied to `ConnectedStreams` which represent two `DataStream`s of possibly different types. `ConnectedStreams` can be created by calling the `connect(otherDataStream)` method of a `DataStream`.
 
-#### Map on ConnectedDataStream
+#### Map on ConnectedStreams
 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:
 
@@ -1032,8 +1032,8 @@ val dataStream2 : DataStream[String] = ...
 </div>
 </div>
 
-#### 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 arbitrarily many values using the Collector interface. 
+#### FlatMap on ConnectedStreams
+The FlatMap operator for `ConnectedStreams` 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.
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -1072,11 +1072,11 @@ val dataStream2 : DataStream[String] = ...
 </div>
 </div>
 
-#### WindowReduce on ConnectedDataStream
+#### WindowReduce on ConnectedStreams
 The windowReduce operator applies a user defined `CoWindowFunction` to time aligned windows of the two data streams and return zero or more elements of an arbitrary type. The user can define the window and slide intervals and can also implement custom timestamps to be used for calculating windows.
 
-#### Reduce on ConnectedDataStream
-The Reduce operator for the `ConnectedDataStream` applies a group-reduce transformation on the grouped joined data streams and then maps the reduced elements to a common output type. It works only for connected data streams where the inputs are grouped.
+#### Reduce on ConnectedStreams
+The Reduce operator for `ConnectedStreams` applies a group-reduce transformation on the grouped joined data streams and then maps the reduced elements to a common output type. It works only for connected data streams where the inputs are grouped.
 
 ### Output splitting
 <div class="codetabs" markdown="1">
@@ -1188,7 +1188,7 @@ To use this functionality the user needs to add the maxWaitTimeMillis parameter
 By default the partitioning of the feedback stream will be automatically set to be the same as the input of the iteration head. To override this the user can set an optional boolean flag in the `closeWith` method. 
 
 #### Iteration head as a co-operator
-The user can also treat the input and feedback stream of a streaming iteration as a `ConnectedDataStream`. This can be used to distinguish the feedback tuples and also to change the type of the iteration feedback. 
+The user can also treat the input and feedback stream of a streaming iteration as `ConnectedStreams`. This can be used to distinguish the feedback tuples and also to change the type of the iteration feedback.
 
 To use this feature the user needs to call the `withFeedbackType(type)` method of the iterative data stream and pass the type of the feedback stream:
 
@@ -1224,13 +1224,13 @@ To use this functionality the user needs to add the maxWaitTimeMillis parameter
 By default the partitioning of the feedback stream will be automatically set to be the same as the input of the iteration head. To override this the user can set an optional boolean flag in the `iterate` method. 
 
 #### Iteration head as a co-operator
-The user can also treat the input and feedback stream of a streaming iteration as a `ConnectedDataStream`. This can be used to distinguish the feedback tuples and also to change the type of the iteration feedback. 
+The user can also treat the input and feedback stream of a streaming iteration as `ConnectedStreams`. This can be used to distinguish the feedback tuples and also to change the type of the iteration feedback.
 
-To use this feature the user needs to call implement a step function that operates on a `ConnectedDataStream` and pass it to the `iterate(…)` call.
+To use this feature the user needs to call implement a step function that operates on `ConnectedStreams` and pass it to the `iterate(…)` call.
 
 {% highlight scala %}
 val iteratedStream = someDataStream.iterate(
-			stepFunction: ConnectedDataStream[T, F] => (DataStream[F], DataStream[R]), 
+			stepFunction: ConnectedStreams[T, F] => (DataStream[F], DataStream[R]),
 			maxWaitTimeMillis)
 {% endhighlight %}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
index f630bce..45da6eb 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -29,7 +29,7 @@
 //		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
 //
 //		@SuppressWarnings("unused")
-//		DataStream<String> dataStream1 = env.addSource(
+//		DataStream<String> inputStream1 = env.addSource(
 //				new FlumeSource<String>("localhost", 41414, new SimpleStringSchema())).addSink(
 //				new FlumeSink<String>("localhost", 42424, new StringToByteSerializer()));
 //

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
deleted file mode 100644
index 0406e35..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
+++ /dev/null
@@ -1,502 +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 org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
-import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
-import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce;
-import org.apache.flink.streaming.api.operators.co.CoStreamMap;
-import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
-import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
-import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-/**
- * The ConnectedDataStream represents a stream for two different data types. It
- * can be used to apply transformations like {@link CoMapFunction} on two
- * {@link DataStream}s
- * 
- * @param <IN1>
- *            Type of the first input data steam.
- * @param <IN2>
- *            Type of the second input data stream.
- */
-public class ConnectedDataStream<IN1, IN2> {
-
-	protected StreamExecutionEnvironment environment;
-	protected DataStream<IN1> dataStream1;
-	protected DataStream<IN2> dataStream2;
-
-	protected boolean isGrouped;
-	protected KeySelector<IN1, ?> keySelector1;
-	protected KeySelector<IN2, ?> keySelector2;
-
-	protected ConnectedDataStream(StreamExecutionEnvironment env, DataStream<IN1> input1, DataStream<IN2> input2) {
-		this.environment = env;
-		if (input1 != null) {
-			this.dataStream1 = input1;
-		}
-		if (input2 != null) {
-			this.dataStream2 = input2;
-		}
-
-		if ((input1 instanceof GroupedDataStream) && (input2 instanceof GroupedDataStream)) {
-			this.isGrouped = true;
-			this.keySelector1 = ((GroupedDataStream<IN1, ?>) input1).keySelector;
-			this.keySelector2 = ((GroupedDataStream<IN2, ?>) input2).keySelector;
-		} else {
-			this.isGrouped = false;
-			this.keySelector1 = null;
-			this.keySelector2 = null;
-		}
-	}
-
-	protected ConnectedDataStream(ConnectedDataStream<IN1, IN2> coDataStream) {
-		this.environment = coDataStream.environment;
-		this.dataStream1 = coDataStream.getFirst();
-		this.dataStream2 = coDataStream.getSecond();
-		this.isGrouped = coDataStream.isGrouped;
-		this.keySelector1 = coDataStream.keySelector1;
-		this.keySelector2 = coDataStream.keySelector2;
-	}
-
-	public <F> F clean(F f) {
-		if (getExecutionEnvironment().getConfig().isClosureCleanerEnabled()) {
-			ClosureCleaner.clean(f, true);
-		}
-		ClosureCleaner.ensureSerializable(f);
-		return f;
-	}
-
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return environment;
-	}
-
-	/**
-	 * Returns the first {@link DataStream}.
-	 * 
-	 * @return The first DataStream.
-	 */
-	public DataStream<IN1> getFirst() {
-		return dataStream1;
-	}
-
-	/**
-	 * Returns the second {@link DataStream}.
-	 * 
-	 * @return The second DataStream.
-	 */
-	public DataStream<IN2> getSecond() {
-		return dataStream2;
-	}
-
-	/**
-	 * Gets the type of the first input
-	 * 
-	 * @return The type of the first input
-	 */
-	public TypeInformation<IN1> getType1() {
-		return dataStream1.getType();
-	}
-
-	/**
-	 * Gets the type of the second input
-	 * 
-	 * @return The type of the second input
-	 */
-	public TypeInformation<IN2> getType2() {
-		return dataStream2.getType();
-	}
-
-	/**
-	 * 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
-	 * {@link ConnectedDataStream#reduce}
-	 * 
-	 * @param keyPosition1
-	 *            The field used to compute the hashcode of the elements in the
-	 *            first input stream.
-	 * @param keyPosition2
-	 *            The field used to compute the hashcode of the elements in the
-	 *            second input stream.
-	 * @return The grouped {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
-		return new ConnectedDataStream<IN1, IN2>(this.environment, dataStream1.groupBy(keyPosition1),
-				dataStream2.groupBy(keyPosition2));
-	}
-
-	/**
-	 * GroupBy operation for connected data stream. Groups the elements of
-	 * input1 and input2 according to keyPositions1 and keyPositions2. Used for
-	 * applying function on grouped data streams for example
-	 * {@link ConnectedDataStream#reduce}
-	 * 
-	 * @param keyPositions1
-	 *            The fields used to group the first input stream.
-	 * @param keyPositions2
-	 *            The fields used to group the second input stream.
-	 * @return The grouped {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> groupBy(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(keyPositions1),
-				dataStream2.groupBy(keyPositions2));
-	}
-
-	/**
-	 * GroupBy operation for connected data stream using key expressions. Groups
-	 * the elements of input1 and input2 according to field1 and field2. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field1
-	 *            The grouping expression for the first input
-	 * @param field2
-	 *            The grouping expression for the second input
-	 * @return The grouped {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> groupBy(String field1, String field2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(field1),
-				dataStream2.groupBy(field2));
-	}
-
-	/**
-	 * GroupBy operation for connected data stream using key expressions. Groups
-	 * the elements of input1 and input2 according to fields1 and fields2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}S underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 * .
-	 * 
-	 * @param fields1
-	 *            The grouping expressions for the first input
-	 * @param fields2
-	 *            The grouping expressions for the second input
-	 * @return The grouped {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> groupBy(String[] fields1, String[] fields2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(fields1),
-				dataStream2.groupBy(fields2));
-	}
-
-	/**
-	 * GroupBy operation for connected data stream. Groups the elements of
-	 * input1 and input2 using keySelector1 and keySelector2. Used for applying
-	 * function on grouped data streams for example
-	 * {@link ConnectedDataStream#reduce}
-	 * 
-	 * @param keySelector1
-	 *            The {@link KeySelector} used for grouping the first input
-	 * @param keySelector2
-	 *            The {@link KeySelector} used for grouping the second input
-	 * @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> groupBy(KeySelector<IN1, ?> keySelector1,
-			KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.groupBy(keySelector1),
-				dataStream2.groupBy(keySelector2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 according to keyPosition1 and keyPosition2.
-	 *
-	 * @param keyPosition1
-	 *            The field used to compute the hashcode of the elements in the
-	 *            first input stream.
-	 * @param keyPosition2
-	 *            The field used to compute the hashcode of the elements in the
-	 *            second input stream.
-	 * @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> partitionByHash(int keyPosition1, int keyPosition2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keyPosition1),
-				dataStream2.partitionByHash(keyPosition2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 according to keyPositions1 and keyPositions2.
-	 *
-	 * @param keyPositions1
-	 *            The fields used to group the first input stream.
-	 * @param keyPositions2
-	 *            The fields used to group the second input stream.
-	 * @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> partitionByHash(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keyPositions1),
-				dataStream2.partitionByHash(keyPositions2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream using key expressions. Partitions
-	 * the elements of input1 and input2 according to field1 and field2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 *
-	 * @param field1
-	 *            The partitioning expressions for the first input
-	 * @param field2
-	 *            The partitioning expressions for the second input
-	 * @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> partitionByHash(String field1, String field2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(field1),
-				dataStream2.partitionByHash(field2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream using key expressions. Partitions
-	 * the elements of input1 and input2 according to fields1 and fields2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 *
-	 * @param fields1
-	 *            The partitioning expressions for the first input
-	 * @param fields2
-	 *            The partitioning expressions for the second input
-	 * @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> partitionByHash(String[] fields1, String[] fields2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(fields1),
-				dataStream2.partitionByHash(fields2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 using keySelector1 and keySelector2.
-	 *
-	 * @param keySelector1
-	 *            The {@link KeySelector} used for partitioning the first input
-	 * @param keySelector2
-	 *            The {@link KeySelector} used for partitioning the second input
-	 * @return @return The partitioned {@link ConnectedDataStream}
-	 */
-	public ConnectedDataStream<IN1, IN2> partitionByHash(KeySelector<IN1, ?> keySelector1,
-														KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedDataStream<IN1, IN2>(environment, dataStream1.partitionByHash(keySelector1),
-				dataStream2.partitionByHash(keySelector2));
-	}
-
-	/**
-	 * 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.
-	 * 
-	 * @param coMapper
-	 *            The CoMapFunction used to jointly transform the two input
-	 *            DataStreams
-	 * @return The transformed {@link DataStream}
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
-
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coMapper,
-				CoMapFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Map", outTypeInfo, new CoStreamMap<IN1, IN2, OUT>(
-				clean(coMapper)));
-
-	}
-
-	/**
-	 * Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
-	 * maps the output to a common type. The transformation calls a
-	 * {@link CoFlatMapFunction#flatMap1} for each element of the first input
-	 * and {@link CoFlatMapFunction#flatMap2} for each element of the second
-	 * input. Each CoFlatMapFunction call returns any number of elements
-	 * including none.
-	 * 
-	 * @param coFlatMapper
-	 *            The CoFlatMapFunction used to jointly transform the two input
-	 *            DataStreams
-	 * @return The transformed {@link DataStream}
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> flatMap(
-			CoFlatMapFunction<IN1, IN2, OUT> coFlatMapper) {
-
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coFlatMapper,
-				CoFlatMapFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Flat Map", outTypeInfo, new CoStreamFlatMap<IN1, IN2, OUT>(
-				clean(coFlatMapper)));
-	}
-
-	/**
-	 * Applies a reduce transformation on a grouped{@link ConnectedDataStream} 
-	 * and maps the outputs to a common type. The reducer is applied on every 
-	 * group of elements sharing the same key. 
-	 * 
-	 * @param coReducer
-	 *            The {@link CoReduceFunction} that will be called for every
-	 *            element of the inputs.
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> reduce(CoReduceFunction<IN1, IN2, OUT> coReducer) {
-
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coReducer,
-				CoReduceFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Reduce", outTypeInfo, getReduceOperator(clean(coReducer)));
-
-	}
-
-	/**
-	 * Applies a CoWindow transformation on the connected DataStreams. The
-	 * transformation calls the {@link CoWindowFunction#coWindow} method for for
-	 * time aligned windows of the two data streams. System time is used as
-	 * default to compute windows.
-	 * 
-	 * @param coWindowFunction
-	 *            The {@link CoWindowFunction} that will be applied for the time
-	 *            windows.
-	 * @param windowSize
-	 *            Size of the windows that will be aligned for both streams in
-	 *            milliseconds.
-	 * @param slideInterval
-	 *            After every function call the windows will be slid by this
-	 *            interval.
-	 * 
-	 * @return The transformed {@link DataStream}.
-	 */
-	@SuppressWarnings("unchecked")
-	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduce(
-			CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize, long slideInterval) {
-		return windowReduce(coWindowFunction, windowSize, slideInterval,
-				(TimestampWrapper<IN1>) SystemTimestamp.getWrapper(),
-				(TimestampWrapper<IN2>) SystemTimestamp.getWrapper());
-	}
-
-	/**
-	 * Applies a CoWindow transformation on the connected DataStreams. The
-	 * transformation calls the {@link CoWindowFunction#coWindow} method for
-	 * time aligned windows of the two data streams. The user can implement
-	 * their own time stamps or use the system time by default.
-	 * 
-	 * @param coWindowFunction
-	 *            The {@link CoWindowFunction} that will be applied for the time
-	 *            windows.
-	 * @param windowSize
-	 *            Size of the windows that will be aligned for both streams. If
-	 *            system time is used it is milliseconds. User defined time
-	 *            stamps are assumed to be monotonically increasing.
-	 * @param slideInterval
-	 *            After every function call the windows will be slid by this
-	 *            interval.
-	 * 
-	 * @param timestamp1
-	 *            User defined time stamps for the first input.
-	 * @param timestamp2
-	 *            User defined time stamps for the second input.
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduce(
-			CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize, long slideInterval,
-			TimestampWrapper<IN1> timestamp1, TimestampWrapper<IN2> timestamp2) {
-
-		if (windowSize < 1) {
-			throw new IllegalArgumentException("Window size must be positive");
-		}
-		if (slideInterval < 1) {
-			throw new IllegalArgumentException("Slide interval must be positive");
-		}
-		
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coWindowFunction,
-				CoWindowFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Window", outTypeInfo, new CoStreamWindow<IN1, IN2, OUT>(
-				clean(coWindowFunction), windowSize, slideInterval, timestamp1, timestamp2));
-
-	}
-
-	protected <OUT> TwoInputStreamOperator<IN1, IN2, OUT> getReduceOperator(
-			CoReduceFunction<IN1, IN2, OUT> coReducer) {
-		if (isGrouped) {
-			return new CoStreamGroupedReduce<IN1, IN2, OUT>(clean(coReducer), keySelector1,
-					keySelector2);
-		} else {
-			throw new UnsupportedOperationException(
-					"Reduce can only be applied on grouped streams.");
-		}
-	}
-
-	public <OUT> SingleOutputStreamOperator<OUT, ?> addGeneralWindowCombine(
-			CoWindowFunction<IN1, IN2, OUT> coWindowFunction, TypeInformation<OUT> outTypeInfo,
-			long windowSize, long slideInterval, TimestampWrapper<IN1> timestamp1,
-			TimestampWrapper<IN2> timestamp2) {
-
-		if (windowSize < 1) {
-			throw new IllegalArgumentException("Window size must be positive");
-		}
-		if (slideInterval < 1) {
-			throw new IllegalArgumentException("Slide interval must be positive");
-		}
-
-		return transform("Co-Window", outTypeInfo, new CoStreamWindow<IN1, IN2, OUT>(
-				clean(coWindowFunction), windowSize, slideInterval, timestamp1, timestamp2));
-
-	}
-
-	public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
-			TypeInformation<OUT> outTypeInfo, TwoInputStreamOperator<IN1, IN2, OUT> operator) {
-
-		// read the output type of the input Transforms to coax out errors about MissinTypeInfo
-		dataStream1.getType();
-		dataStream2.getType();
-
-		TwoInputTransformation<IN1, IN2, OUT> transform = new TwoInputTransformation<IN1, IN2, OUT>(
-				dataStream1.getTransformation(),
-				dataStream2.getTransformation(),
-				functionName,
-				operator,
-				outTypeInfo,
-				environment.getParallelism());
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(environment, transform);
-
-		getExecutionEnvironment().addOperator(transform);
-
-		return returnStream;
-	}
-
-	protected ConnectedDataStream<IN1, IN2> copy() {
-		return new ConnectedDataStream<IN1, IN2>(this);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
new file mode 100644
index 0000000..2447c1e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
@@ -0,0 +1,331 @@
+/*
+ * 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.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
+import org.apache.flink.streaming.api.operators.co.CoStreamMap;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+
+/**
+ * {@code ConnectedStreams} represents two connected streams of (possible) different data types. It
+ * can be used to apply transformations such as {@link CoMapFunction} on two
+ * {@link DataStream DataStreams}
+ * 
+ * @param <IN1> Type of the first input data steam.
+ * @param <IN2> Type of the second input data stream.
+ */
+public class ConnectedStreams<IN1, IN2> {
+
+	protected StreamExecutionEnvironment environment;
+	protected DataStream<IN1> inputStream1;
+	protected DataStream<IN2> inputStream2;
+
+	protected ConnectedStreams(StreamExecutionEnvironment env,
+			DataStream<IN1> input1,
+			DataStream<IN2> input2) {
+		this.environment = env;
+		if (input1 != null) {
+			this.inputStream1 = input1;
+		}
+		if (input2 != null) {
+			this.inputStream2 = input2;
+		}
+	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return environment;
+	}
+
+	/**
+	 * Returns the first {@link DataStream}.
+	 *
+	 * @return The first DataStream.
+	 */
+	public DataStream<IN1> getFirstInput() {
+		return inputStream1;
+	}
+
+	/**
+	 * Returns the second {@link DataStream}.
+	 *
+	 * @return The second DataStream.
+	 */
+	public DataStream<IN2> getSecondInput() {
+		return inputStream2;
+	}
+
+	/**
+	 * Gets the type of the first input
+	 *
+	 * @return The type of the first input
+	 */
+	public TypeInformation<IN1> getType1() {
+		return inputStream1.getType();
+	}
+
+	/**
+	 * Gets the type of the second input
+	 *
+	 * @return The type of the second input
+	 */
+	public TypeInformation<IN2> getType2() {
+		return inputStream2.getType();
+	}
+
+	/**
+	 * GroupBy operation for connected data stream. Groups the elements of
+	 * input1 and input2 according to keyPosition1 and keyPosition2.
+	 *
+	 * @param keyPosition1
+	 *            The field used to compute the hashcode of the elements in the
+	 *            first input stream.
+	 * @param keyPosition2
+	 *            The field used to compute the hashcode of the elements in the
+	 *            second input stream.
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> groupBy(int keyPosition1, int keyPosition2) {
+		return new ConnectedStreams<>(this.environment, inputStream1.groupBy(keyPosition1),
+				inputStream2.groupBy(keyPosition2));
+	}
+
+	/**
+	 * GroupBy operation for connected data stream. Groups the elements of
+	 * input1 and input2 according to keyPositions1 and keyPositions2.
+	 *
+	 * @param keyPositions1
+	 *            The fields used to group the first input stream.
+	 * @param keyPositions2
+	 *            The fields used to group the second input stream.
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> groupBy(int[] keyPositions1, int[] keyPositions2) {
+		return new ConnectedStreams<>(environment, inputStream1.groupBy(keyPositions1),
+				inputStream2.groupBy(keyPositions2));
+	}
+
+	/**
+	 * GroupBy operation for connected data stream using key expressions. Groups
+	 * the elements of input1 and input2 according to field1 and field2. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field1
+	 *            The grouping expression for the first input
+	 * @param field2
+	 *            The grouping expression for the second input
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> groupBy(String field1, String field2) {
+		return new ConnectedStreams<>(environment, inputStream1.groupBy(field1),
+				inputStream2.groupBy(field2));
+	}
+
+	/**
+	 * GroupBy operation for connected data stream using key expressions. Groups
+	 * the elements of input1 and input2 according to fields1 and fields2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}S underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 * .
+	 *
+	 * @param fields1
+	 *            The grouping expressions for the first input
+	 * @param fields2
+	 *            The grouping expressions for the second input
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> groupBy(String[] fields1, String[] fields2) {
+		return new ConnectedStreams<>(environment, inputStream1.groupBy(fields1),
+				inputStream2.groupBy(fields2));
+	}
+
+	/**
+	 * GroupBy operation for connected data stream. Groups the elements of
+	 * input1 and input2 using keySelector1 and keySelector2.
+	 *
+	 * @param keySelector1
+	 *            The {@link KeySelector} used for grouping the first input
+	 * @param keySelector2
+	 *            The {@link KeySelector} used for grouping the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> groupBy(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
+		return new ConnectedStreams<>(environment, inputStream1.groupBy(keySelector1),
+				inputStream2.groupBy(keySelector2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 according to keyPosition1 and keyPosition2.
+	 *
+	 * @param keyPosition1
+	 *            The field used to compute the hashcode of the elements in the
+	 *            first input stream.
+	 * @param keyPosition2
+	 *            The field used to compute the hashcode of the elements in the
+	 *            second input stream.
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(int keyPosition1, int keyPosition2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPosition1),
+				inputStream2.partitionByHash(keyPosition2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 according to keyPositions1 and keyPositions2.
+	 *
+	 * @param keyPositions1
+	 *            The fields used to group the first input stream.
+	 * @param keyPositions2
+	 *            The fields used to group the second input stream.
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(int[] keyPositions1, int[] keyPositions2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPositions1),
+				inputStream2.partitionByHash(keyPositions2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream using key expressions. Partitions
+	 * the elements of input1 and input2 according to field1 and field2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 *
+	 * @param field1
+	 *            The partitioning expressions for the first input
+	 * @param field2
+	 *            The partitioning expressions for the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(String field1, String field2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(field1),
+				inputStream2.partitionByHash(field2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream using key expressions. Partitions
+	 * the elements of input1 and input2 according to fields1 and fields2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 *
+	 * @param fields1
+	 *            The partitioning expressions for the first input
+	 * @param fields2
+	 *            The partitioning expressions for the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(String[] fields1, String[] fields2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(fields1),
+				inputStream2.partitionByHash(fields2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 using keySelector1 and keySelector2.
+	 *
+	 * @param keySelector1
+	 *            The {@link KeySelector} used for partitioning the first input
+	 * @param keySelector2
+	 *            The {@link KeySelector} used for partitioning the second input
+	 * @return @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keySelector1),
+				inputStream2.partitionByHash(keySelector2));
+	}
+
+	/**
+	 * Applies a CoMap transformation on a {@link ConnectedStreams} 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.
+	 * 
+	 * @param coMapper The CoMapFunction used to jointly transform the two input DataStreams
+	 * @return The transformed {@link DataStream}
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
+
+		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coMapper,
+				CoMapFunction.class, false, true, getType1(), getType2(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Co-Map", outTypeInfo, new CoStreamMap<>(inputStream1.clean(coMapper)));
+
+	}
+
+	/**
+	 * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
+	 * maps the output to a common type. The transformation calls a
+	 * {@link CoFlatMapFunction#flatMap1} for each element of the first input
+	 * and {@link CoFlatMapFunction#flatMap2} for each element of the second
+	 * input. Each CoFlatMapFunction call returns any number of elements
+	 * including none.
+	 * 
+	 * @param coFlatMapper
+	 *            The CoFlatMapFunction used to jointly transform the two input
+	 *            DataStreams
+	 * @return The transformed {@link DataStream}
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> flatMap(
+			CoFlatMapFunction<IN1, IN2, OUT> coFlatMapper) {
+
+		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coFlatMapper,
+				CoFlatMapFunction.class, false, true, getType1(), getType2(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Co-Flat Map", outTypeInfo, new CoStreamFlatMap<>(inputStream1.clean(coFlatMapper)));
+	}
+
+	public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
+			TypeInformation<OUT> outTypeInfo,
+			TwoInputStreamOperator<IN1, IN2, OUT> operator) {
+
+		// read the output type of the input Transforms to coax out errors about MissingTypeInfo
+		inputStream1.getType();
+		inputStream2.getType();
+
+		TwoInputTransformation<IN1, IN2, OUT> transform = new TwoInputTransformation<>(
+				inputStream1.getTransformation(),
+				inputStream2.getTransformation(),
+				functionName,
+				operator,
+				outTypeInfo,
+				environment.getParallelism());
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(environment, transform);
+
+		getExecutionEnvironment().addOperator(transform);
+
+		return returnStream;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index ad159f9..3389016 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -44,7 +44,6 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.temporal.StreamCrossOperator;
 import org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.TimestampExtractor;
@@ -218,17 +217,17 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Creates a new {@link ConnectedDataStream} by connecting
+	 * Creates a new {@link ConnectedStreams} by connecting
 	 * {@link DataStream} outputs of (possible) different types with each other.
 	 * The DataStreams connected using this operator can be used with
 	 * CoFunctions to apply joint transformations.
 	 * 
 	 * @param dataStream
 	 *            The DataStream with which this stream will be connected.
-	 * @return The {@link ConnectedDataStream}.
+	 * @return The {@link ConnectedStreams}.
 	 */
-	public <R> ConnectedDataStream<T, R> connect(DataStream<R> dataStream) {
-		return new ConnectedDataStream<T, R>(environment, this, dataStream);
+	public <R> ConnectedStreams<T, R> connect(DataStream<R> dataStream) {
+		return new ConnectedStreams<T, R>(environment, this, dataStream);
 	}
 
 	/**
@@ -536,7 +535,7 @@ public class DataStream<T> {
 	 * the data stream that will be fed back and used as the input for the
 	 * iteration head. The user can also use different feedback type than the
 	 * input of the iteration and treat the input and feedback streams as a
-	 * {@link ConnectedDataStream} be calling
+	 * {@link ConnectedStreams} be calling
 	 * {@link IterativeDataStream#withFeedbackType(TypeInformation)}
 	 * <p>
 	 * A common usage pattern for streaming iterations is to use output
@@ -567,7 +566,7 @@ public class DataStream<T> {
 	 * the data stream that will be fed back and used as the input for the
 	 * iteration head. The user can also use different feedback type than the
 	 * input of the iteration and treat the input and feedback streams as a
-	 * {@link ConnectedDataStream} be calling
+	 * {@link ConnectedStreams} be calling
 	 * {@link IterativeDataStream#withFeedbackType(TypeInformation)}
 	 * <p>
 	 * A common usage pattern for streaming iterations is to use output
@@ -680,32 +679,6 @@ public class DataStream<T> {
 		return new StreamProjection<T>(this, fieldIndexes).projectTupleX();
 	}
 
-
-	/**
-	 * Initiates a temporal Cross transformation.<br/>
-	 * A Cross transformation combines the elements of two {@link DataStream}s
-	 * into one DataStream over a specified time window. It builds all pair
-	 * combinations of elements of both DataStreams, i.e., it builds a Cartesian
-	 * product.
-	 * 
-	 * <p>
-	 * This method returns a {@link StreamCrossOperator} on which the
-	 * {@link StreamCrossOperator#onWindow} should be called to define the
-	 * window.
-	 * <p>
-	 * Call {@link StreamCrossOperator.CrossWindow#with(org.apache.flink.api.common.functions.CrossFunction)}
-	 * to define a custom cross function.
-	 * 
-	 * @param dataStreamToCross
-	 *            The other DataStream with which this DataStream is crossed.
-	 * @return A {@link StreamCrossOperator} to continue the definition of the
-	 *         cross transformation.
-	 * 
-	 */
-	public <IN2> StreamCrossOperator<T, IN2> cross(DataStream<IN2> dataStreamToCross) {
-		return new StreamCrossOperator<T, IN2>(this, dataStreamToCross);
-	}
-
 	/**
 	 * Initiates a temporal Join transformation. <br/>
 	 * A temporal Join transformation joins the elements of two

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
index 6b12013..2fe3848 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeDataStream.java
@@ -81,56 +81,56 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
 	/**
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedDataStream}.
+	 * {@link ConnectedStreams}.
 	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
 	 * @param feedbackTypeString
 	 *            String describing the type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStream}.
+	 * @return A {@link ConnectedIterativeDataStreams}.
 	 */
-	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(String feedbackTypeString) {
+	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(String feedbackTypeString) {
 		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
 	}
 
 	/**
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedDataStream}.
+	 * {@link ConnectedStreams}.
 	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
 	 * @param feedbackTypeClass
 	 *            Class of the elements in the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStream}.
+	 * @return A {@link ConnectedIterativeDataStreams}.
 	 */
-	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
+	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
 		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
 	}
 
 	/**
 	 * Changes the feedback type of the iteration and allows the user to apply
 	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedDataStream}.
+	 * {@link ConnectedStreams}.
 	 *
 	 * <p>
 	 * For type safety the user needs to define the feedback type
 	 * 
 	 * @param feedbackType
 	 *            The type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeDataStream}.
+	 * @return A {@link ConnectedIterativeDataStreams}.
 	 */
-	public <F> ConnectedIterativeDataStream<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
-		return new ConnectedIterativeDataStream<T, F>(originalInput, feedbackType, maxWaitTime);
+	public <F> ConnectedIterativeDataStreams<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
+		return new ConnectedIterativeDataStreams<T, F>(originalInput, feedbackType, maxWaitTime);
 	}
 	
 	/**
-	 * The {@link ConnectedIterativeDataStream} represent a start of an
+	 * The {@link ConnectedIterativeDataStreams} represent a start of an
 	 * iterative part of a streaming program, where the original input of the
 	 * iteration and the feedback of the iteration are connected as in a
-	 * {@link ConnectedDataStream}.
+	 * {@link ConnectedStreams}.
 	 *
 	 * <p>
 	 * The user can distinguish between the two inputs using co-transformation,
@@ -142,24 +142,26 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
 	 * @param <F>
 	 *            Type of the feedback of the iteration
 	 */
-	public static class ConnectedIterativeDataStream<I, F> extends ConnectedDataStream<I, F>{
+	public static class ConnectedIterativeDataStreams<I, F> extends ConnectedStreams<I, F> {
 
 		private CoFeedbackTransformation<F> coFeedbackTransformation;
 
-		public ConnectedIterativeDataStream(DataStream<I> input, TypeInformation<F> feedbackType, long waitTime) {
+		public ConnectedIterativeDataStreams(DataStream<I> input,
+				TypeInformation<F> feedbackType,
+				long waitTime) {
 			super(input.getExecutionEnvironment(),
 					input,
 					new DataStream<F>(input.getExecutionEnvironment(),
 							new CoFeedbackTransformation<F>(input.getParallelism(),
 									feedbackType,
 									waitTime)));
-			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecond().getTransformation();
+			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecondInput().getTransformation();
 		}
 
 		/**
 		 * Closes the iteration. This method defines the end of the iterative
 		 * program part that will be fed back to the start of the iteration as
-		 * the second input in the {@link ConnectedDataStream}.
+		 * the second input in the {@link ConnectedStreams}.
 		 * 
 		 * @param feedbackStream
 		 *            {@link DataStream} that will be used as second input to
@@ -186,34 +188,34 @@ public class IterativeDataStream<T> extends SingleOutputStreamOperator<T, Iterat
 				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
 		
 		@Override
-		public ConnectedDataStream<I, F> groupBy(int keyPosition1, int keyPosition2) {throw groupingException;}
+		public ConnectedStreams<I, F> groupBy(int keyPosition1, int keyPosition2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> groupBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+		public ConnectedStreams<I, F> groupBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> groupBy(String field1, String field2) {throw groupingException;}
+		public ConnectedStreams<I, F> groupBy(String field1, String field2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> groupBy(String[] fields1, String[] fields2) {throw groupingException;}
+		public ConnectedStreams<I, F> groupBy(String[] fields1, String[] fields2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> groupBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
+		public ConnectedStreams<I, F> groupBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
+		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+		public ConnectedStreams<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
+		public ConnectedStreams<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
+		public ConnectedStreams<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
 		
 		@Override
-		public ConnectedDataStream<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
+		public ConnectedStreams<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
 		
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
deleted file mode 100644
index e0aafb7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamCrossOperator.java
+++ /dev/null
@@ -1,115 +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.temporal;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.operators.CrossOperator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.functions.co.CrossWindowFunction;
-
-public class StreamCrossOperator<I1, I2> extends
-		TemporalOperator<I1, I2, StreamCrossOperator.CrossWindow<I1, I2, Tuple2<I1, I2>>> {
-
-	public StreamCrossOperator(DataStream<I1> input1, DataStream<I2> input2) {
-		super(input1, input2);
-	}
-
-	protected <F> F clean(F f) {
-		if (input1.getExecutionEnvironment().getConfig().isClosureCleanerEnabled()) {
-			ClosureCleaner.clean(f, true);
-		}
-		ClosureCleaner.ensureSerializable(f);
-		return f;
-	}
-
-	@Override
-	protected CrossWindow<I1, I2, Tuple2<I1, I2>> createNextWindowOperator() {
-
-		CrossWindowFunction<I1, I2, Tuple2<I1, I2>> crossWindowFunction = new CrossWindowFunction<I1, I2, Tuple2<I1, I2>>(
-				clean(new CrossOperator.DefaultCrossFunction<I1, I2>()));
-
-		return new CrossWindow<I1, I2, Tuple2<I1, I2>>(this, input1.connect(input2).addGeneralWindowCombine(
-				crossWindowFunction,
-				new TupleTypeInfo<Tuple2<I1, I2>>(input1.getType(), input2.getType()), windowSize,
-				slideInterval, timeStamp1, timeStamp2));
-	}
-
-	public static class CrossWindow<I1, I2, R> extends
-			SingleOutputStreamOperator<R, CrossWindow<I1, I2, R>> implements
-			TemporalWindow<CrossWindow<I1, I2, R>> {
-
-		private StreamCrossOperator<I1, I2> op;
-
-		public CrossWindow(StreamCrossOperator<I1, I2> op, DataStream<R> ds) {
-			super(ds.getExecutionEnvironment(), ds.getTransformation());
-			this.op = op;
-		}
-
-		public CrossWindow<I1, I2, R> every(long length, TimeUnit timeUnit) {
-			return every(timeUnit.toMillis(length));
-		}
-
-		@SuppressWarnings("unchecked")
-		public CrossWindow<I1, I2, R> every(long length) {
-
-			CrossWindowFunction<I1, I2, Tuple2<I1, I2>> crossWindowFunction = new CrossWindowFunction<I1, I2, Tuple2<I1, I2>>(
-					clean(new CrossOperator.DefaultCrossFunction<I1, I2>()));
-
-			return (CrossWindow<I1, I2, R>) new CrossWindow<I1, I2, Tuple2<I1, I2>>(op, op.input1.connect(op.input2).addGeneralWindowCombine(
-					crossWindowFunction,
-					new TupleTypeInfo<Tuple2<I1, I2>>(op.input1.getType(), op.input2.getType()), op.windowSize,
-					length, op.timeStamp1, op.timeStamp2));
-		}
-
-		/**
-		 * Finalizes a temporal Cross transformation by applying a
-		 * {@link CrossFunction} to each pair of crossed elements.<br/>
-		 * Each CrossFunction call returns exactly one element.
-		 * 
-		 * @param function
-		 *            The CrossFunction that is called for each pair of crossed
-		 *            elements.
-		 * @return The crossed data streams
-		 * 
-		 */
-		@SuppressWarnings("unchecked")
-		public <R> SingleOutputStreamOperator<R, ?> with(CrossFunction<I1, I2, R> function) {
-			TypeInformation<R> outTypeInfo = TypeExtractor.getCrossReturnTypes(function,
-					op.input1.getType(), op.input2.getType());
-
-			CrossWindowFunction<I1, I2, R> crossWindowFunction = new CrossWindowFunction<I1, I2, R>(clean(function));
-
-			return new CrossWindow<I1, I2, R>(op, op.input1.connect(op.input2).addGeneralWindowCombine(
-					crossWindowFunction,
-					outTypeInfo, op.windowSize,
-					op.slideInterval, op.timeStamp1, op.timeStamp2));
-
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
index e48d707..999d197 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
@@ -26,11 +26,9 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.Keys;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.functions.co.JoinWindowFunction;
 import org.apache.flink.streaming.util.keys.KeySelectorUtil;
 
 public class StreamJoinOperator<I1, I2> extends
@@ -204,19 +202,20 @@ public class StreamJoinOperator<I1, I2> extends
 
 		private JoinedStream<I1, I2, Tuple2<I1, I2>> createJoinOperator() {
 
-			JoinFunction<I1, I2, Tuple2<I1, I2>> joinFunction = new DefaultJoinFunction<I1, I2>();
-
-			JoinWindowFunction<I1, I2, Tuple2<I1, I2>> joinWindowFunction = getJoinWindowFunction(
-					joinFunction, this);
-
-			TypeInformation<Tuple2<I1, I2>> outType = new TupleTypeInfo<Tuple2<I1, I2>>(
-					op.input1.getType(), op.input2.getType());
-
-			return new JoinedStream<I1, I2, Tuple2<I1, I2>>(this, op.input1
-					.groupBy(keys1)
-					.connect(op.input2.groupBy(keys2))
-					.addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
-							op.slideInterval, op.timeStamp1, op.timeStamp2));
+//			JoinFunction<I1, I2, Tuple2<I1, I2>> joinFunction = new DefaultJoinFunction<I1, I2>();
+//
+//			JoinWindowFunction<I1, I2, Tuple2<I1, I2>> joinWindowFunction = getJoinWindowFunction(
+//					joinFunction, this);
+//
+//			TypeInformation<Tuple2<I1, I2>> outType = new TupleTypeInfo<Tuple2<I1, I2>>(
+//					op.input1.getType(), op.input2.getType());
+
+//			return new JoinedStream<I1, I2, Tuple2<I1, I2>>(this, op.input1
+//					.groupBy(keys1)
+//					.connect(op.input2.groupBy(keys2))
+//					.addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
+//							op.slideInterval, op.timeStamp1, op.timeStamp2));
+			return null;
 		}
 
 		public static class JoinedStream<I1, I2, R> extends
@@ -240,15 +239,16 @@ public class StreamJoinOperator<I1, I2> extends
 				TypeInformation<OUT> outType = TypeExtractor.getJoinReturnTypes(joinFunction,
 						predicate.op.input1.getType(), predicate.op.input2.getType());
 
-				JoinWindowFunction<I1, I2, OUT> joinWindowFunction = getJoinWindowFunction(joinFunction, predicate);
+//				JoinWindowFunction<I1, I2, OUT> joinWindowFunction = getJoinWindowFunction(joinFunction, predicate);
+//
 
-
-				return new JoinedStream<I1, I2, OUT>(
-						predicate, predicate.op.input1
-						.groupBy(predicate.keys1)
-						.connect(predicate.op.input2.groupBy(predicate.keys2))
-						.addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize,
-								predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2));
+//				return new JoinedStream<I1, I2, OUT>(
+//						predicate, predicate.op.input1
+//						.groupBy(predicate.keys1)
+//						.connect(predicate.op.input2.groupBy(predicate.keys2))
+//						.addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize,
+//								predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2));
+				return null;
 			}
 		}
 	}
@@ -267,8 +267,8 @@ public class StreamJoinOperator<I1, I2> extends
 		}
 	}
 
-	private static <I1, I2, OUT> JoinWindowFunction<I1, I2, OUT> getJoinWindowFunction(
-			JoinFunction<I1, I2, OUT> joinFunction, JoinPredicate<I1, I2> predicate) {
-		return new JoinWindowFunction<I1, I2, OUT>(predicate.keys1, predicate.keys2, joinFunction);
-	}
+//	private static <I1, I2, OUT> JoinWindowFunction<I1, I2, OUT> getJoinWindowFunction(
+//			JoinFunction<I1, I2, OUT> joinFunction, JoinPredicate<I1, I2> predicate) {
+//		return new JoinWindowFunction<I1, I2, OUT>(predicate.keys1, predicate.keys2, joinFunction);
+//	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoReduceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoReduceFunction.java
deleted file mode 100644
index 9518d4b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoReduceFunction.java
+++ /dev/null
@@ -1,107 +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.functions.co;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-
-/**
- * The CoReduceFunction interface represents a Reduce transformation with two
- * different input streams. The reduce1 function combine groups of elements of
- * the first input with the same key to a single value, while reduce2 combine
- * groups of elements of the second input with the same key to a single value.
- * Each produced values are mapped to the same type by map1 and map2,
- * respectively, to form one output stream.
- * 
- * The basic syntax for using a grouped ReduceFunction is as follows:
- * 
- * <pre>
- * <blockquote>
- * ConnectedDataStream<X> input = ...;
- * 
- * ConnectedDataStream<X> result = input.groupBy(keyPosition1, keyPosition2)
- *          .reduce(new MyCoReduceFunction(), keyPosition1, keyPosition2).addSink(...);
- * </blockquote>
- * </pre>
- * <p>
- * 
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public interface CoReduceFunction<IN1, IN2, OUT> extends Function, Serializable {
-
-	/**
-	 * The core method of CoReduceFunction, combining two values of the first
-	 * input into one value of the same type. The reduce1 function is
-	 * consecutively applied to all values of a group until only a single value
-	 * remains.
-	 *
-	 * @param value1
-	 *            The first value to combine.
-	 * @param value2
-	 *            The second value to combine.
-	 * @return The combined value of both input values.
-	 *
-	 * @throws Exception
-	 *             This method may throw exceptions. Throwing an exception will
-	 *             cause the operation to fail and may trigger recovery.
-	 */
-	IN1 reduce1(IN1 value1, IN1 value2) throws Exception;
-
-	/**
-	 * The core method of ReduceFunction, combining two values of the second
-	 * input into one value of the same type. The reduce2 function is
-	 * consecutively applied to all values of a group until only a single value
-	 * remains.
-	 *
-	 * @param value1
-	 *            The first value to combine.
-	 * @param value2
-	 *            The second value to combine.
-	 * @return The combined value of both input values.
-	 *
-	 * @throws Exception
-	 *             This method may throw exceptions. Throwing an exception will
-	 *             cause the operation to fail and may trigger recovery.
-	 */
-	IN2 reduce2(IN2 value1, IN2 value2) throws Exception;
-
-	/**
-	 * Maps the reduced first input to the output type.
-	 * 
-	 * @param value
-	 *            Type of the first input.
-	 * @return the output type.
-	 */
-	OUT map1(IN1 value) throws Exception;
-
-	/**
-	 * Maps the reduced second input to the output type.
-	 * 
-	 * @param value
-	 *            Type of the second input.
-	 * @return the output type.
-	 */
-	OUT map2(IN2 value) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoWindowFunction.java
deleted file mode 100644
index 1ae5137..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoWindowFunction.java
+++ /dev/null
@@ -1,30 +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.functions.co;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.util.Collector;
-
-public interface CoWindowFunction<IN1, IN2, O> extends Function, Serializable {
-
-	void coWindow(List<IN1> first, List<IN2> second, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CrossWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CrossWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CrossWindowFunction.java
deleted file mode 100644
index e9c0169..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CrossWindowFunction.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.co;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.util.Collector;
-
-public class CrossWindowFunction<IN1, IN2, OUT> implements CoWindowFunction<IN1, IN2, OUT> {
-	private static final long serialVersionUID = 1L;
-
-	private CrossFunction<IN1, IN2, OUT> crossFunction;
-
-	public CrossWindowFunction(CrossFunction<IN1, IN2, OUT> crossFunction) {
-		this.crossFunction = crossFunction;
-	}
-
-	@Override
-	public void coWindow(List<IN1> first, List<IN2> second, Collector<OUT> out) throws Exception {
-		for (IN1 firstValue : first) {
-			for (IN2 secondValue : second) {
-				out.collect(crossFunction.cross(firstValue, secondValue));
-			}
-		}
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/JoinWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/JoinWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/JoinWindowFunction.java
deleted file mode 100644
index 6f658c6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/JoinWindowFunction.java
+++ /dev/null
@@ -1,77 +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.functions.co;
-
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.util.Collector;
-
-public class JoinWindowFunction<IN1, IN2, OUT> implements CoWindowFunction<IN1, IN2, OUT> {
-	private static final long serialVersionUID = 1L;
-
-	private KeySelector<IN1, ?> keySelector1;
-	private KeySelector<IN2, ?> keySelector2;
-	private JoinFunction<IN1, IN2, OUT> joinFunction;
-
-	public JoinWindowFunction(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2,
-			JoinFunction<IN1, IN2, OUT> joinFunction) {
-		this.keySelector1 = keySelector1;
-		this.keySelector2 = keySelector2;
-		this.joinFunction = joinFunction;
-	}
-
-	@Override
-	public void coWindow(List<IN1> first, List<IN2> second, Collector<OUT> out) throws Exception {
-
-		Map<Object, List<IN1>> map = build(first);
-
-		for (IN2 record : second) {
-			Object key = keySelector2.getKey(record);
-			List<IN1> match = map.get(key);
-			if (match != null) {
-				for (IN1 matching : match) {
-					out.collect(joinFunction.join(matching, record));
-				}
-			}
-		}
-
-	}
-
-	private Map<Object, List<IN1>> build(List<IN1> records) throws Exception {
-
-		Map<Object, List<IN1>> map = new HashMap<Object, List<IN1>>();
-
-		for (IN1 record : records) {
-			Object key = keySelector1.getKey(record);
-			List<IN1> current = map.get(key);
-			if (current == null) {
-				current = new LinkedList<IN1>();
-				map.put(key, current);
-			}
-			current.add(record);
-		}
-
-		return map;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoReduceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoReduceFunction.java
deleted file mode 100644
index d3e6f3a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoReduceFunction.java
+++ /dev/null
@@ -1,40 +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.functions.co;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * A RichCoReduceFunction represents a Reduce transformation with two different input
- * types. In addition to that the user can use the features provided by the
- * {@link RichFunction} interface.
- *
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public abstract class RichCoReduceFunction<IN1, IN2, OUT> extends AbstractRichFunction implements
-		CoReduceFunction<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoWindowFunction.java
deleted file mode 100644
index e317065..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoWindowFunction.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.functions.co;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.util.Collector;
-
-public abstract class RichCoWindowFunction<IN1, IN2, O> extends AbstractRichFunction implements
-		CoWindowFunction<IN1, IN2, O> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public abstract void coWindow(List<IN1> first, List<IN2> second, Collector<O> out)
-			throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/23d8e264/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java
deleted file mode 100644
index b46a929..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamGroupedReduce.java
+++ /dev/null
@@ -1,77 +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.operators.co;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class CoStreamGroupedReduce<IN1, IN2, OUT> extends CoStreamReduce<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected KeySelector<IN1, ?> keySelector1;
-	protected KeySelector<IN2, ?> keySelector2;
-	private Map<Object, IN1> values1;
-	private Map<Object, IN2> values2;
-	IN1 reduced1;
-	IN2 reduced2;
-
-	public CoStreamGroupedReduce(CoReduceFunction<IN1, IN2, OUT> coReducer,
-			KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
-		super(coReducer);
-		this.keySelector1 = keySelector1;
-		this.keySelector2 = keySelector2;
-		values1 = new HashMap<Object, IN1>();
-		values2 = new HashMap<Object, IN2>();
-	}
-
-	@Override
-	public void processElement1(StreamRecord<IN1> elementRecord) throws Exception {
-		IN1 element = elementRecord.getValue();
-		Object key = keySelector1.getKey(element);
-		currentValue1 = values1.get(key);
-		if (currentValue1 != null) {
-			reduced1 = userFunction.reduce1(currentValue1, element);
-			values1.put(key, reduced1);
-			output.collect(elementRecord.replace(userFunction.map1(reduced1)));
-		} else {
-			values1.put(key, element);
-			output.collect(elementRecord.replace(userFunction.map1(element)));
-		}
-	}
-
-	@Override
-	public void processElement2(StreamRecord<IN2> elementRecord) throws Exception {
-		IN2 element = elementRecord.getValue();
-
-		Object key = keySelector2.getKey(element);
-		currentValue2 = values2.get(key);
-		if (currentValue2 != null) {
-			reduced2 = userFunction.reduce2(currentValue2, element);
-			values2.put(key, reduced2);
-			output.collect(elementRecord.replace(userFunction.map2(reduced2)));
-		} else {
-			values2.put(key, element);
-			output.collect(elementRecord.replace(userFunction.map2(element)));
-		}
-	}
-}


[03/13] flink git commit: Add Scala API for new Windowing

Posted by al...@apache.org.
Add Scala API for new Windowing

This adds window/timeWindow to KeyedStream along with windowAll/timeWindowAll
on DataStream.

The added API classes are AllWindowedStream and WindowedStream.

This also adds Translations tests similar to those for the Java API:
 - AllWindowTranslationTest.scala
 - WindowTranslationTest.scala


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

Branch: refs/heads/master
Commit: d0c9e78fdbcfe60db4bcfc9e6c2d4ba70fa00935
Parents: 9baadfe
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Oct 1 21:23:56 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       |   4 +
 .../api/datastream/WindowedStream.java          |   5 +
 .../streaming/api/scala/AllWindowedStream.scala | 123 ++++++++++++
 .../flink/streaming/api/scala/DataStream.scala  |  84 +++++++-
 .../flink/streaming/api/scala/KeyedStream.scala |  91 ++++++++-
 .../streaming/api/scala/WindowedStream.scala    | 126 ++++++++++++
 .../api/scala/AllWindowTranslationTest.scala    | 192 +++++++++++++++++++
 .../StreamingScalaAPICompletenessTest.scala     |  24 +++
 .../api/scala/WindowTranslationTest.scala       | 185 ++++++++++++++++++
 9 files changed, 830 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index e5c7c18..134029f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -228,4 +229,7 @@ public class AllWindowedStream<T, W extends Window> {
 		return null;
 	}
 
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return input.getExecutionEnvironment();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 16898dd..41adab5 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -299,4 +300,8 @@ public class WindowedStream<T, K, W extends Window> {
 
 		return null;
 	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return input.getExecutionEnvironment();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
new file mode 100644
index 0000000..4f36722
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -0,0 +1,123 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream}
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
+import org.apache.flink.streaming.api.windowing.evictors.Evictor
+import org.apache.flink.streaming.api.windowing.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.windows.Window
+
+import scala.reflect.ClassTag
+
+/**
+ * A [[AllWindowedStream]] represents a data stream where the stream of
+ * elements is split into windows based on a
+ * [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]. Window emission
+ * is triggered based on a [[Trigger]].
+ *
+ * If an [[Evictor]] is specified it will be
+ * used to evict elements from the window after
+ * evaluation was triggered by the [[Trigger]] but before the actual evaluation of the window.
+ * When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ *
+ * Note that the [[AllWindowedStream()]] is purely and API construct, during runtime
+ * the [[AllWindowedStream()]] will be collapsed together with the
+ * operation over the window into one single operation.
+ *
+ * @tparam T The type of elements in the stream.
+ * @tparam W The type of [[Window]] that the
+ *           [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]
+ *           assigns the elements to.
+ */
+class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
+
+  /**
+   * Sets the [[Trigger]] that should be used to trigger window emission.
+   */
+  def trigger(trigger: Trigger[_ >: T, _ >: W]): AllWindowedStream[T, W] = {
+    javaStream.trigger(trigger)
+    this
+  }
+
+  /**
+   * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
+   *
+   * Note: When using an evictor window performance will degrade significantly, since
+   * pre-aggregation of window results cannot be used.
+   */
+  def evictor(evictor: Evictor[_ >: T, _ >: W]): AllWindowedStream[T, W] = {
+    javaStream.evictor(evictor)
+    this
+  }
+
+  // ------------------------------------------------------------------------
+  //  Operations on the keyed windows
+  // ------------------------------------------------------------------------
+  /**
+   * Applies a reduce function to the window. The window function is called for each evaluation
+   * of the window for each key individually. The output of the reduce function is interpreted
+   * as a regular non-windowed stream.
+   *
+   * This window will try and pre-aggregate data as much as the window policies permit. For example,
+   * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+   * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide
+   * interval, so a few elements are stored per key (one per slide interval).
+   * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+   * aggregation tree.
+   *
+   * @param function The reduce function.
+   * @return The data stream that is the result of applying the reduce function to the window.
+   */
+  def reduceWindow(function: ReduceFunction[T]): DataStream[T] = {
+    javaStream.reduceWindow(clean(function))
+  }
+
+  /**
+   * Applies the given window function to each window. The window function is called for each
+   * evaluation of the window for each key individually. The output of the window function is
+   * interpreted as a regular non-windowed stream.
+   *
+   * Not that this function requires that all data in the windows is buffered until the window
+   * is evaluated, as the function provides no means of pre-aggregation.
+   *
+   * @param function The window function.
+   * @return The data stream that is the result of applying the window function to the window.
+   */
+  def apply[R: TypeInformation: ClassTag](function: AllWindowFunction[T, R, W]): DataStream[R] = {
+    javaStream.apply(clean(function), implicitly[TypeInformation[R]])
+  }
+
+
+  // ------------------------------------------------------------------------
+  //  Utilities
+  // ------------------------------------------------------------------------
+
+  /**
+   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+   */
+  private[flink] def clean[F <: AnyRef](f: F): F = {
+    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 07828db..7dfaeef 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -19,6 +19,10 @@
 package org.apache.flink.streaming.api.scala
 
 import org.apache.flink.streaming.api.functions.{AscendingTimestampExtractor, TimestampExtractor}
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.{ProcessingTime, EventTime, AbstractTime}
+import org.apache.flink.streaming.api.windowing.windows.{Window, TimeWindow}
+import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWindowedStream}
 
 import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
@@ -31,7 +35,7 @@ import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.scala.operators.ScalaCsvOutputFormat
 import org.apache.flink.core.fs.{FileSystem, Path}
 import org.apache.flink.streaming.api.collector.selector.OutputSelector
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, DataStreamSink, SingleOutputStreamOperator, KeyedDataStream}
+import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, DataStreamSink, SingleOutputStreamOperator}
 import org.apache.flink.streaming.api.functions.sink.SinkFunction
 import org.apache.flink.streaming.api.windowing.helper.WindowingHelper
 import org.apache.flink.streaming.api.windowing.policy.{EvictionPolicy, TriggerPolicy}
@@ -610,6 +614,82 @@ class DataStream[T](javaStream: JavaStream[T]) {
     javaStream.every(windowingHelper)
 
   /**
+   * Windows this DataStream into tumbling time windows.
+   *
+   * This is a shortcut for either `.window(TumblingTimeWindows.of(size))` or
+   * `.window(TumblingProcessingTimeWindows.of(size))` depending on the time characteristic
+   * set using
+   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic]].
+   *
+   * @param size The size of the window.
+   */
+  def timeWindowAll(size: AbstractTime): AllWindowedStream[T, TimeWindow] = {
+    val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
+    val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+
+    actualSize match {
+      case t: EventTime =>
+        val assigner = TumblingTimeWindows.of(actualSize.toMilliseconds)
+          .asInstanceOf[WindowAssigner[T, TimeWindow]]
+        windowAll(assigner)
+      case t: ProcessingTime =>
+        val assigner = TumblingProcessingTimeWindows.of(actualSize.toMilliseconds)
+          .asInstanceOf[WindowAssigner[T, TimeWindow]]
+        windowAll(assigner)
+      case _ => throw new RuntimeException("Invalid time: " + actualSize)
+    }
+  }
+
+  /**
+   * Windows this DataStream into sliding time windows.
+   *
+   * This is a shortcut for either `.window(SlidingTimeWindows.of(size, slide))` or
+   * `.window(SlidingProcessingTimeWindows.of(size, slide))` depending on the time characteristic
+   * set using
+   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic]].
+   *
+   * @param size The size of the window.
+   */
+  def timeWindowAll(size: AbstractTime, slide: AbstractTime): AllWindowedStream[T, TimeWindow] = {
+    val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
+    val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+    val actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+
+    actualSize match {
+      case t: EventTime =>
+        val assigner = SlidingTimeWindows.of(
+          actualSize.toMilliseconds,
+          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+        windowAll(assigner)
+      case t: ProcessingTime =>
+        val assigner = SlidingProcessingTimeWindows.of(
+          actualSize.toMilliseconds,
+          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+        windowAll(assigner)
+      case _ => throw new RuntimeException("Invalid time: " + actualSize)
+    }
+  }
+
+  /**
+   * Windows this data stream to a [[AllWindowedStream]], which evaluates windows
+   * over a key grouped stream. Elements are put into windows by a [[WindowAssigner]]. The grouping
+   * of elements is done both by key and by window.
+   *
+   * A [[org.apache.flink.streaming.api.windowing.triggers.Trigger]] can be defined to specify
+   * when windows are evaluated. However, `WindowAssigner` have a default `Trigger`
+   * that is used if a `Trigger` is not specified.
+   *
+   * Note: This operation can be inherently non-parallel since all elements have to pass through
+   * the same operator instance. (Only for special cases, such as aligned time windows is
+   * it possible to perform this operation in parallel).
+   *
+   * @param assigner The `WindowAssigner` that assigns elements to windows.
+   * @return The trigger windows data stream.
+   */
+  def windowAll[W <: Window](assigner: WindowAssigner[_ >: T, W]): AllWindowedStream[T, W] = {
+    new AllWindowedStream[T, W](new JavaAllWindowedStream[T, W](javaStream, assigner))
+  }
+  /**
    * Extracts a timestamp from an element and assigns it as the internal timestamp of that element.
    * The internal timestamps are, for example, used to to event-time window operations.
    *
@@ -780,7 +860,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
 
   /**
    * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
+   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
    */
   private[flink] def clean[F <: AnyRef](f: F): F = {
     new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index 25244cd..232e4bb 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -18,11 +18,14 @@
 
 package org.apache.flink.streaming.api.scala
 
-import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream, DataStream => JavaStream }
+import org.apache.flink.streaming.api.datastream.{KeyedStream => KeyedJavaStream, DataStream => JavaStream, WindowedStream => WindowedJavaStream}
 import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
 import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
 import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
 import org.apache.flink.streaming.api.operators.StreamGroupedReduce
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.{ProcessingTime, EventTime, AbstractTime}
+import org.apache.flink.streaming.api.windowing.windows.{Window, TimeWindow}
 import scala.reflect.ClassTag
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.functions.FoldFunction
@@ -30,7 +33,91 @@ import org.apache.flink.api.common.functions.ReduceFunction
 
 
 class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T](javaStream) {
- 
+
+  // ------------------------------------------------------------------------
+  //  Windowing
+  // ------------------------------------------------------------------------
+
+  /**
+   * Windows this [[KeyedStream]] into tumbling time windows.
+   *
+   * <p>
+   * This is a shortcut for either `.window(TumblingTimeWindows.of(size))` or
+   * `.window(TumblingProcessingTimeWindows.of(size))` depending on the time characteristic
+   * set using
+   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic()]]
+   *
+   * @param size The size of the window.
+   */
+  def timeWindow(size: AbstractTime): WindowedStream[T, K, TimeWindow] = {
+    val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
+    val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+
+    actualSize match {
+      case t: EventTime =>
+        val assigner = TumblingTimeWindows.of(actualSize.toMilliseconds)
+          .asInstanceOf[WindowAssigner[T, TimeWindow]]
+        window(assigner)
+      case t: ProcessingTime =>
+        val assigner = TumblingProcessingTimeWindows.of(actualSize.toMilliseconds)
+          .asInstanceOf[WindowAssigner[T, TimeWindow]]
+        window(assigner)
+      case _ => throw new RuntimeException("Invalid time: " + actualSize)
+    }
+  }
+
+  /**
+   * Windows this [[KeyedStream]] into sliding time windows.
+   *
+   * <p>
+   * This is a shortcut for either `.window(SlidingTimeWindows.of(size))` or
+   * `.window(SlidingProcessingTimeWindows.of(size))` depending on the time characteristic
+   * set using
+   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic()]]
+   *
+   * @param size The size of the window.
+   */
+  def timeWindow(size: AbstractTime, slide: AbstractTime): WindowedStream[T, K, TimeWindow] = {
+    val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
+    val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+    val actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
+
+    actualSize match {
+      case t: EventTime =>
+        val assigner = SlidingTimeWindows.of(
+          actualSize.toMilliseconds,
+          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+        window(assigner)
+      case t: ProcessingTime =>
+        val assigner = SlidingProcessingTimeWindows.of(
+          actualSize.toMilliseconds,
+          actualSlide.toMilliseconds).asInstanceOf[WindowAssigner[T, TimeWindow]]
+        window(assigner)
+      case _ => throw new RuntimeException("Invalid time: " + actualSize)
+    }
+  }
+
+  /**
+   * Windows this data stream to a [[WindowedStream]], which evaluates windows
+   * over a key grouped stream. Elements are put into windows by a [[WindowAssigner]]. The
+   * grouping of elements is done both by key and by window.
+   *
+   * <p>
+   * A [[org.apache.flink.streaming.api.windowing.triggers.Trigger]] can be defined to specify
+   * when windows are evaluated. However, `WindowAssigner` have a default `Trigger`
+   * that is used if a `Trigger` is not specified.
+   *
+   * @param assigner The `WindowAssigner` that assigns elements to windows.
+   * @return The trigger windows data stream.
+   */
+  def window[W <: Window](assigner: WindowAssigner[_ >: T, W]): WindowedStream[T, K, W] = {
+    new WindowedStream(new WindowedJavaStream[T, K, W](javaStream, assigner))
+  }
+
+  // ------------------------------------------------------------------------
+  //  Non-Windowed aggregation operations
+  // ------------------------------------------------------------------------
+
   /**
    * Creates a new [[DataStream]] by reducing the elements of this DataStream
    * using an associative reduce function. An independent aggregate is kept per key.

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
new file mode 100644
index 0000000..a688846
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream}
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction
+import org.apache.flink.streaming.api.windowing.evictors.Evictor
+import org.apache.flink.streaming.api.windowing.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.windows.Window
+
+import scala.reflect.ClassTag
+
+/**
+ * A [[WindowedStream]] represents a data stream where elements are grouped by
+ * key, and for each key, the stream of elements is split into windows based on a
+ * [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]. Window emission
+ * is triggered based on a [[Trigger]].
+ *
+ * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
+ * different points for each key.
+ *
+ * If an [[org.apache.flink.streaming.api.windowing.evictors.Evictor]] is specified it will
+ * be used to evict elements from the window after evaluation was triggered by the [[Trigger]]
+ * but before the actual evaluation of the window. When using an evictor window performance will
+ * degrade significantly, since pre-aggregation of window results cannot be used.
+ *
+ * Note that the [[WindowedStream]] is purely and API construct, during runtime
+ * the [[WindowedStream]] will be collapsed together with the
+ * [[KeyedStream]] and the operation over the window into one single operation.
+ *
+ * @tparam T The type of elements in the stream.
+ * @tparam K The type of the key by which elements are grouped.
+ * @tparam W The type of [[Window]] that the
+ *           [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]
+ *           assigns the elements to.
+ */
+class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
+
+  /**
+   * Sets the [[Trigger]] that should be used to trigger window emission.
+   */
+  def trigger(trigger: Trigger[_ >: T, _ >: W]): WindowedStream[T, K, W] = {
+    javaStream.trigger(trigger)
+    this
+  }
+
+  /**
+   * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
+   *
+   * Note: When using an evictor window performance will degrade significantly, since
+   * pre-aggregation of window results cannot be used.
+   */
+  def evictor(evictor: Evictor[_ >: T, _ >: W]): WindowedStream[T, K, W] = {
+    javaStream.evictor(evictor)
+    this
+  }
+
+  // ------------------------------------------------------------------------
+  //  Operations on the keyed windows
+  // ------------------------------------------------------------------------
+  /**
+   * Applies a reduce function to the window. The window function is called for each evaluation
+   * of the window for each key individually. The output of the reduce function is interpreted
+   * as a regular non-windowed stream.
+   *
+   * This window will try and pre-aggregate data as much as the window policies permit. For example,
+   * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+   * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide
+   * interval, so a few elements are stored per key (one per slide interval).
+   * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+   * aggregation tree.
+   *
+   * @param function The reduce function.
+   * @return The data stream that is the result of applying the reduce function to the window.
+   */
+  def reduceWindow(function: ReduceFunction[T]): DataStream[T] = {
+    javaStream.reduceWindow(clean(function))
+  }
+
+  /**
+   * Applies the given window function to each window. The window function is called for each
+   * evaluation of the window for each key individually. The output of the window function is
+   * interpreted as a regular non-windowed stream.
+   *
+   * Not that this function requires that all data in the windows is buffered until the window
+   * is evaluated, as the function provides no means of pre-aggregation.
+   *
+   * @param function The window function.
+   * @return The data stream that is the result of applying the window function to the window.
+   */
+  def apply[R: TypeInformation: ClassTag](function: WindowFunction[T, R, K, W]): DataStream[R] = {
+    javaStream.apply(clean(function), implicitly[TypeInformation[R]])
+  }
+
+
+  // ------------------------------------------------------------------------
+  //  Utilities
+  // ------------------------------------------------------------------------
+
+  /**
+   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+   */
+  private[flink] def clean[F <: AnyRef](f: F): F = {
+    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
new file mode 100644
index 0000000..35c7fcc
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
@@ -0,0 +1,192 @@
+/*
+ * 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.scala
+
+
+import org.apache.flink.api.common.functions.RichReduceFunction
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction}
+import org.apache.flink.streaming.api.transformations.OneInputTransformation
+import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
+import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
+import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, WatermarkTrigger, CountTrigger}
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
+import org.apache.flink.streaming.runtime.operators.windowing._
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.util.Collector
+
+import org.junit.Assert._
+import org.junit.{Ignore, Test}
+
+class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
+
+  /**
+   * These tests ensure that the fast aligned time windows operator is used if the
+   * conditions are right.
+   *
+   * TODO: update once we have optimized aligned time windows operator for all-windows
+   */
+  @Ignore
+  @Test
+  def testFastTimeWindows(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[AggregatingProcessingTimeWindowOperator[_, _]])
+
+    val window2 = source
+      .keyBy(0)
+      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
+      def apply(
+                    tuple: Tuple,
+                    window: TimeWindow,
+                    values: java.lang.Iterable[(String, Int)],
+                    out: Collector[(String, Int)]) { }
+    })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[AccumulatingProcessingTimeWindowOperator[_, _, _]])
+  }
+
+  @Test
+  def testNonEvicting(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+      .trigger(CountTrigger.of(100))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
+    val winOperator1 = operator1.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
+    assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(
+      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
+
+
+    val window2 = source
+      .windowAll(TumblingProcessingTimeWindows.of(1000))
+      .trigger(CountTrigger.of(100))
+      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
+      def apply(
+                    window: TimeWindow,
+                    values: java.lang.Iterable[(String, Int)],
+                    out: Collector[(String, Int)]) { }
+    })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
+    val winOperator2 = operator2.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
+    assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+  }
+
+  @Test
+  def testEvicting(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+      .evictor(TimeEvictor.of(1000))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]])
+    val winOperator1 = operator1.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
+    assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
+    assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+
+
+    val window2 = source
+      .windowAll(TumblingProcessingTimeWindows.of(1000))
+      .trigger(CountTrigger.of(100))
+      .evictor(CountEvictor.of(1000))
+      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
+      def apply(
+                    window: TimeWindow,
+                    values: java.lang.Iterable[(String, Int)],
+                    out: Collector[(String, Int)]) { }
+    })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]])
+    val winOperator2 = operator2.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
+    assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+  }
+}
+
+// ------------------------------------------------------------------------
+//  UDFs
+// ------------------------------------------------------------------------
+
+class DummyReducer extends RichReduceFunction[(String, Int)] {
+  def reduce(value1: (String, Int), value2: (String, Int)): (String, Int) = {
+    value1
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index 66fe197..6ecdb85 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -44,6 +44,9 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "org.apache.flink.streaming.api.datastream.DataStream.getTransformation",
       "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.copy",
       "org.apache.flink.streaming.api.datastream.ConnectedStreams.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getFirstInput",
+      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getSecondInput",
       "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType1",
       "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType2",
       "org.apache.flink.streaming.api.datastream.ConnectedStreams.addGeneralWindowCombine",
@@ -51,6 +54,12 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getType",
       "org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig",
 
+      "org.apache.flink.streaming.api.datastream.WindowedStream.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.AllWindowedStream.getExecutionEnvironment",
+
+      "org.apache.flink.streaming.api.datastream.KeyedStream.transform",
+      "org.apache.flink.streaming.api.datastream.KeyedStream.getKeySelector",
+
       "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.isChainingEnabled",
       "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." +
         "getStateHandleProvider",
@@ -114,6 +123,21 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       classOf[SplitDataStream[_]])
 
     checkMethods(
+      "WindowedStream", "WindowedStream",
+      classOf[org.apache.flink.streaming.api.datastream.WindowedStream[_, _, _]],
+      classOf[WindowedStream[_, _, _]])
+
+    checkMethods(
+      "AllWindowedStream", "AllWindowedStream",
+      classOf[org.apache.flink.streaming.api.datastream.AllWindowedStream[_, _]],
+      classOf[AllWindowedStream[_, _]])
+
+    checkMethods(
+      "KeyedStream", "KeyedStream",
+      classOf[org.apache.flink.streaming.api.datastream.KeyedStream[_, _]],
+      classOf[KeyedStream[_, _]])
+
+    checkMethods(
       "StreamJoinOperator", "StreamJoinOperator",
       classOf[org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator[_,_]],
       classOf[StreamJoinOperator[_,_]])

http://git-wip-us.apache.org/repos/asf/flink/blob/d0c9e78f/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
new file mode 100644
index 0000000..49d0a1a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
@@ -0,0 +1,185 @@
+/*
+ * 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.scala
+
+
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction
+import org.apache.flink.streaming.api.transformations.OneInputTransformation
+import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
+import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
+import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
+import org.apache.flink.streaming.runtime.operators.windowing.{EvictingWindowOperator, WindowOperator, AccumulatingProcessingTimeWindowOperator, AggregatingProcessingTimeWindowOperator}
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.util.Collector
+
+import org.junit.Assert._
+import org.junit.Test
+
+class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
+
+  /**
+   * These tests ensure that the fast aligned time windows operator is used if the
+   * conditions are right.
+   */
+  @Test
+  def testFastTimeWindows(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .keyBy(0)
+      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+        .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+    
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[AggregatingProcessingTimeWindowOperator[_, _]])
+
+    val window2 = source
+      .keyBy(0)
+      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
+        def apply(
+            key: Tuple,
+            window: TimeWindow,
+            values: java.lang.Iterable[(String, Int)],
+            out: Collector[(String, Int)]) { }
+      })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[AccumulatingProcessingTimeWindowOperator[_, _, _]])
+  }
+
+  @Test
+  def testNonEvicting(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .keyBy(0)
+      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .trigger(CountTrigger.of(100))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
+    val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
+    assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(
+      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
+
+
+    val window2 = source
+      .keyBy(0)
+      .window(TumblingProcessingTimeWindows.of(1000))
+      .trigger(CountTrigger.of(100))
+      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
+      def apply(
+                    tuple: Tuple,
+                    window: TimeWindow,
+                    values: java.lang.Iterable[(String, Int)],
+                    out: Collector[(String, Int)]) { }
+    })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
+    val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
+    assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+  }
+
+  @Test
+  def testEvicting(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    val source = env.fromElements(("hello", 1), ("hello", 2))
+
+    val reducer = new DummyReducer
+
+    val window1 = source
+      .keyBy(0)
+      .window(SlidingProcessingTimeWindows.of(1000, 100))
+      .evictor(TimeEvictor.of(1000))
+      .reduceWindow(reducer)
+
+    val transform1 = window1.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator1 = transform1.getOperator
+
+    assertTrue(operator1.isInstanceOf[EvictingWindowOperator[_, _, _, _]])
+    val winOperator1 = operator1.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
+    assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
+    assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+
+
+    val window2 = source
+      .keyBy(0)
+      .window(TumblingProcessingTimeWindows.of(1000))
+      .trigger(CountTrigger.of(100))
+      .evictor(CountEvictor.of(1000))
+      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
+      def apply(
+                    tuple: Tuple,
+                    window: TimeWindow,
+                    values: java.lang.Iterable[(String, Int)],
+                    out: Collector[(String, Int)]) { }
+    })
+
+    val transform2 = window2.getJavaStream.getTransformation
+      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
+
+    val operator2 = transform2.getOperator
+
+    assertTrue(operator2.isInstanceOf[EvictingWindowOperator[_, _, _, _]])
+    val winOperator2 = operator2.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
+    assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
+    assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
+  }
+}


[13/13] flink git commit: [FLINK-2807] Add Javadocs for new windowing semantics/internals

Posted by al...@apache.org.
[FLINK-2807] Add Javadocs for new windowing semantics/internals


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

Branch: refs/heads/master
Commit: 62df0a0349b276d4a5b7d9954d2a07f367a61d2d
Parents: 8c2c769
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Oct 3 16:47:28 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Oct 5 16:36:35 2015 +0200

----------------------------------------------------------------------
 .../api/windowing/assigners/GlobalWindows.java  |  8 +++
 .../assigners/SlidingProcessingTimeWindows.java | 13 ++++
 .../windowing/assigners/SlidingTimeWindows.java | 13 ++++
 .../TumblingProcessingTimeWindows.java          | 13 ++++
 .../assigners/TumblingTimeWindows.java          | 13 ++++
 .../api/windowing/assigners/WindowAssigner.java | 22 ++++++
 .../api/windowing/evictors/CountEvictor.java    | 10 +++
 .../api/windowing/evictors/DeltaEvictor.java    | 16 +++++
 .../api/windowing/evictors/Evictor.java         | 25 ++++++-
 .../api/windowing/evictors/TimeEvictor.java     | 11 +++
 .../ContinuousProcessingTimeTrigger.java        | 12 ++++
 .../triggers/ContinuousWatermarkTrigger.java    | 14 ++++
 .../api/windowing/triggers/CountTrigger.java    | 11 +++
 .../api/windowing/triggers/DeltaTrigger.java    | 20 ++++++
 .../triggers/ProcessingTimeTrigger.java         |  7 ++
 .../api/windowing/triggers/PurgingTrigger.java  | 15 +++++
 .../api/windowing/triggers/Trigger.java         | 70 ++++++++++++++++++--
 .../windowing/triggers/WatermarkTrigger.java    |  9 +++
 .../operators/BucketStreamSortOperator.java     | 16 ++++-
 .../runtime/operators/Triggerable.java          |  7 +-
 .../EvictingNonKeyedWindowOperator.java         | 10 ++-
 .../windowing/EvictingWindowOperator.java       | 12 ++++
 .../windowing/NonKeyedWindowOperator.java       |  9 +++
 .../operators/windowing/WindowOperator.java     | 60 +++++++++++++++--
 .../windowing/buffers/EvictingWindowBuffer.java | 15 ++++-
 .../windowing/buffers/HeapWindowBuffer.java     |  8 ++-
 .../buffers/PreAggregatingHeapWindowBuffer.java |  9 ++-
 .../windowing/buffers/WindowBuffer.java         | 38 +++++++++--
 .../windowing/buffers/WindowBufferFactory.java  | 24 +++++++
 29 files changed, 483 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
index 391a6a4..52c8f55 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
@@ -23,6 +23,14 @@ import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import java.util.Collection;
 import java.util.Collections;
 
+/**
+ * A {@link WindowAssigner} that assigns all elements to the same global window.
+ *
+ * <p>
+ * Use this if you want to use a {@link Trigger} and
+ * {@link org.apache.flink.streaming.api.windowing.evictors.Evictor} to to flexible, policy based
+ * windows.
+ */
 public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
index 6fc79b0..65d7641 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
@@ -28,6 +28,19 @@ import java.io.ObjectInputStream;
 import java.util.Collection;
 import java.util.List;
 
+/**
+ * A {@link WindowAssigner} that windows elements into sliding, time-based windows. The windowing
+ * is based on system time. Windows can possibly overlap.
+ *
+ * <p>
+ * For example, in order to window into windows of 1 minute, every 10 seconds:
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> in = ...;
+ * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
+ * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
+ *   keyed.window(SlidingProcessingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
+ * } </pre>
+ */
 public class SlidingProcessingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
index 49bff05..52ae356 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
@@ -26,6 +26,19 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+/**
+ * A {@link WindowAssigner} that windows elements into sliding windows based on the timestamp of the
+ * elements. Windows can possibly overlap.
+ *
+ * <p>
+ * For example, in order to window into windows of 1 minute, every 10 seconds:
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> in = ...;
+ * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
+ * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
+ *   keyed.window(SlidingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
+ * } </pre>
+ */
 public class SlidingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
index 1f2eebf..41f6362 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
@@ -25,6 +25,19 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import java.util.Collection;
 import java.util.Collections;
 
+/**
+ * A {@link WindowAssigner} that windows elements into time-based windows. The windowing is
+ * based on system time. Windows cannot overlap.
+ *
+ * <p>
+ * For example, in order to window into windows of 1 minute, every 10 seconds:
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> in = ...;
+ * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
+ * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
+ *   keyed.window(TumblingProcessingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
+ * } </pre>
+ */
 public class TumblingProcessingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
index 019f45b..b6022b3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
@@ -25,6 +25,19 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import java.util.Collection;
 import java.util.Collections;
 
+/**
+ * A {@link WindowAssigner} that windows elements into windows based on the timestamp of the
+ * elements. Windows cannot overlap.
+ *
+ * <p>
+ * For example, in order to window into windows of 1 minute, every 10 seconds:
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> in = ...;
+ * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
+ * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
+ *   keyed.window(TumblingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
+ * } </pre>
+ */
 public class TumblingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
index 5996426..105caa6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
@@ -23,10 +23,32 @@ import scala.Serializable;
 
 import java.util.Collection;
 
+/**
+ * A {@code WindowAssigner} assigns zero or more {@link Window Windows} to an element.
+ *
+ * <p>
+ * In a window operation, elements are grouped by their key (if available) and by the windows to
+ * which it was assigned. The set of elements with the same key and window is called a pane.
+ * When a {@link Trigger} decides that a certain pane should fire the
+ * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} is applied
+ * to produce output elements for that pane.
+ *
+ * @param <T> The type of elements that this WindowAssigner can assign windows to.
+ * @param <W> The type of {@code Window} that this assigner assigns.
+ */
 public abstract class WindowAssigner<T, W extends Window> implements Serializable {
 	private static final long serialVersionUID = 1L;
 
+	/**
+	 * Returns a {@code Collection} of windows that should be assigned to the element.
+	 *
+	 * @param element The element to which windows should be assigned.
+	 * @param timestamp The timestamp of the element.
+	 */
 	public abstract Collection<W> assignWindows(T element, long timestamp);
 
+	/**
+	 * Returns the default trigger associated with this {@code WindowAssigner}.
+	 */
 	public abstract Trigger<T, W> getDefaultTrigger();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
index 04636ee..0a078e2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
@@ -20,6 +20,11 @@ package org.apache.flink.streaming.api.windowing.evictors;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
+/**
+ * An {@link Evictor} that keeps only a certain amount of elements.
+ *
+ * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
+ */
 public class CountEvictor<W extends Window> implements Evictor<Object, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -38,6 +43,11 @@ public class CountEvictor<W extends Window> implements Evictor<Object, W> {
 		}
 	}
 
+	/**
+	 * Creates a {@code CountEvictor} that keeps the given number of elements.
+	 *
+	 * @param maxCount The number of elements to keep in the pane.
+	 */
 	public static <W extends Window> CountEvictor<W> of(long maxCount) {
 		return new CountEvictor<>(maxCount);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
index c7872ce..0083a04 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
@@ -22,6 +22,16 @@ import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
+/**
+ * An {@link Evictor} that keeps elements based on a {@link DeltaFunction} and a threshold.
+ *
+ * <p>
+ * Eviction starts from the first element of the buffer and removes all elements from the buffer
+ * which have a higher delta then the threshold. As soon as there is an element with a lower delta,
+ * the eviction stops.
+ *
+ * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
+ */
 public class DeltaEvictor<T, W extends Window> implements Evictor<T, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -52,6 +62,12 @@ public class DeltaEvictor<T, W extends Window> implements Evictor<T, W> {
 		return "DeltaEvictor(" +  deltaFunction + ", " + threshold + ")";
 	}
 
+	/**
+	 * Creates a {@code DeltaEvictor} from the given threshold and {@code DeltaFunction}.
+	 *
+	 * @param threshold The threshold
+	 * @param deltaFunction The {@code DeltaFunction}
+	 */
 	public static <T, W extends Window> DeltaEvictor<T, W> of(double threshold, DeltaFunction<T> deltaFunction) {
 		return new DeltaEvictor<>(threshold, deltaFunction);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
index db04ac4..1a6c5c8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
@@ -21,8 +21,31 @@ import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import scala.Serializable;
 
+/**
+ * An {@code Evictor} can remove elements from a pane before it is being processed and after
+ * window evaluation was triggered by a
+ * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ *
+ * <p>
+ * A pane is the bucket of elements that have the same key (assigned by the
+ * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
+ * be in multiple panes of it was assigned to multiple windows by the
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
+ * have their own instance of the {@code Evictor}.
+ *
+ * @param <T> The type of elements that this {@code Evictor} can evict.
+ * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
+ */
 public interface Evictor<T, W extends Window> extends Serializable {
 
-	public abstract int evict(Iterable<StreamRecord<T>> elements, int size, W window);
+	/**
+	 * Computes how many elements should be removed from the pane. The result specifies how
+	 * many elements should be removed from the beginning.
+	 *
+	 * @param elements The elements currently in the pane.
+	 * @param size The current number of elements in the pane.
+	 * @param window The {@link Window}
+	 */
+	int evict(Iterable<StreamRecord<T>> elements, int size, W window);
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
index 2965214..5004c42 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
@@ -22,6 +22,12 @@ import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
+/**
+ * An {@link Evictor} that keeps elements for a certain amount of time. Elements older
+ * than {@code current_time - keep_time} are evicted.
+ *
+ * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
+ */
 public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -55,6 +61,11 @@ public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
 		return windowSize;
 	}
 
+	/**
+	 * Creates a {@code TimeEvictor} that keeps the given number of elements.
+	 *
+	 * @param windowSize The amount of time for which to keep elements.
+	 */
 	public static <W extends Window> TimeEvictor<W> of(AbstractTime windowSize) {
 		return new TimeEvictor<>(windowSize.toMilliseconds());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
index 24e8ce3..f23f6ee 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
@@ -21,6 +21,12 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
+/**
+ * A {@link Trigger} that continuously fires based on a given time interval. The time is the current
+ * system time.
+ *
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
 public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigger<Object, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -80,6 +86,12 @@ public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigge
 		return "ContinuousProcessingTimeTrigger(" + interval + ")";
 	}
 
+	/**
+	 * Creates a trigger that continuously fires based on the given interval.
+	 *
+	 * @param interval The time interval at which to fire.
+	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+	 */
 	public static <W extends Window> ContinuousProcessingTimeTrigger<W> of(AbstractTime interval) {
 		return new ContinuousProcessingTimeTrigger<>(interval.toMilliseconds());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
index e11ceba..02ea81d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
@@ -21,6 +21,14 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
+/**
+ * A {@link Trigger} that continuously fires based on a given time interval. This fires based
+ * on {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks}.
+ *
+ * @see org.apache.flink.streaming.api.watermark.Watermark
+ *
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
 public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Object, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -66,6 +74,12 @@ public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Obj
 		return interval;
 	}
 
+	/**
+	 * Creates a trigger that continuously fires based on the given interval.
+	 *
+	 * @param interval The time interval at which to fire.
+	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+	 */
 	public static <W extends Window> ContinuousWatermarkTrigger<W> of(AbstractTime interval) {
 		return new ContinuousWatermarkTrigger<>(interval.toMilliseconds());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
index a51fae6..53480fe 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
@@ -19,6 +19,11 @@ package org.apache.flink.streaming.api.windowing.triggers;
 
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
+/**
+ * A {@link Trigger} that fires once the count of elements in a pane reaches the given count.
+ *
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
 public class CountTrigger<W extends Window> implements Trigger<Object, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -55,6 +60,12 @@ public class CountTrigger<W extends Window> implements Trigger<Object, W> {
 		return "CountTrigger(" +  maxCount + ")";
 	}
 
+	/**
+	 * Creates a trigger that fires once the number of elements in a pane reaches the given count.
+	 *
+	 * @param maxCount The count of elements at which to fire.
+	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+	 */
 	public static <W extends Window> CountTrigger<W> of(long maxCount) {
 		return new CountTrigger<>(maxCount);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
index ecd7ed0..cf4cf0c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
@@ -20,6 +20,16 @@ package org.apache.flink.streaming.api.windowing.triggers;
 import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
+/**
+ * A {@link Trigger} that fires based on a {@link DeltaFunction} and a threshold.
+ *
+ * <p>
+ * This trigger calculates a delta between the data point which triggered last
+ * and the currently arrived data point. It triggers if the delta is higher than
+ * a specified threshold.
+ *
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
 public class DeltaTrigger<T, W extends Window> implements Trigger<T, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -60,6 +70,16 @@ public class DeltaTrigger<T, W extends Window> implements Trigger<T, W> {
 		return "DeltaTrigger(" +  deltaFunction + ", " + threshold + ")";
 	}
 
+	/**
+	 * Creates a delta trigger from the given threshold and {@code DeltaFunction}.
+	 *
+	 * @param threshold The threshold at which to trigger.
+	 * @param deltaFunction The delta function to use
+	 *
+	 * @param <T> The type of elements on which this trigger can operate.
+	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+	 * @return
+	 */
 	public static <T, W extends Window> DeltaTrigger<T, W> of(double threshold, DeltaFunction<T> deltaFunction) {
 		return new DeltaTrigger<>(threshold, deltaFunction);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
index f693a67..cc3440c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
@@ -19,6 +19,10 @@ package org.apache.flink.streaming.api.windowing.triggers;
 
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 
+/**
+ * A {@link Trigger} that fires once the current system time passes the end of the window
+ * to which a pane belongs.
+ */
 public class ProcessingTimeTrigger implements Trigger<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 
@@ -50,6 +54,9 @@ public class ProcessingTimeTrigger implements Trigger<Object, TimeWindow> {
 		return "ProcessingTimeTrigger()";
 	}
 
+	/**
+	 * Creates a new trigger that fires once system time passes the end of the window.
+	 */
 	public static ProcessingTimeTrigger create() {
 		return new ProcessingTimeTrigger();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
index 88e22cd..1c896a7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
@@ -20,6 +20,16 @@ package org.apache.flink.streaming.api.windowing.triggers;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
+/**
+ * A trigger that can turn any {@link Trigger} into a purging {@code Trigger}.
+ *
+ * <p>
+ * When the nested trigger fires, this will return a {@code FIRE_AND_PURGE}
+ * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerResult}
+ *
+ * @param <T> The type of elements on which this trigger can operate.
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
 public class PurgingTrigger<T, W extends Window> implements Trigger<T, W> {
 	private static final long serialVersionUID = 1L;
 
@@ -65,6 +75,11 @@ public class PurgingTrigger<T, W extends Window> implements Trigger<T, W> {
 		return "PurgingTrigger(" + nestedTrigger.toString() + ")";
 	}
 
+	/**
+	 * Creates a new purging trigger from the given {@code Trigger}.
+	 *
+	 * @param nestedTrigger The trigger that is wrapped by this purging trigger
+	 */
 	public static <T, W extends Window> PurgingTrigger<T, W> of(Trigger<T, W> nestedTrigger) {
 		return new PurgingTrigger<>(nestedTrigger);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
index b04aacf..97d9ba5 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
@@ -20,21 +20,81 @@ package org.apache.flink.streaming.api.windowing.triggers;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import scala.Serializable;
 
+/**
+ * A {@code Trigger} determines when a pane of a window should be evaluated to emit the
+ * results for that part of the window.
+ *
+ * <p>
+ * A pane is the bucket of elements that have the same key (assigned by the
+ * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
+ * be in multiple panes of it was assigned to multiple windows by the
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
+ * have their own instance of the {@code Trigger}.
+ *
+ * @param <T> The type of elements on which this {@code Trigger} works.
+ * @param <W> The type of {@link Window Windows} on which this {@code Trigger} can operate.
+ */
 public interface Trigger<T, W extends Window> extends Serializable {
 
-	public TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx);
+	/**
+	 * Called for every element that gets added to a pane. The result of this will determine
+	 * whether the pane is evaluated to emit results.
+	 *
+	 * @param element The element that arrived.
+	 * @param timestamp The timestamp of the element that arrived.
+	 * @param window The window to which this pane belongs.
+	 * @param ctx A context object that can be used to register timer callbacks.
+	 */
+	TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx);
 
-	public TriggerResult onTime(long time, TriggerContext ctx);
+	/**
+	 * Called when a timer that was set using the trigger context fires.
+	 *
+	 * @param time The timestamp at which the timer fired.
+	 * @param ctx A context object that can be used to register timer callbacks.
+	 */
+	TriggerResult onTime(long time, TriggerContext ctx);
 
-	public Trigger<T, W> duplicate();
+	/**
+	 * Creates a duplicate of the {@code Trigger} without the state of the original {@code Trigger}.
+	 * @return The duplicate {@code Trigger} object.
+	 */
+	Trigger<T, W> duplicate();
 
-	public static enum TriggerResult {
+	/**
+	 * Result type for trigger methods. This determines what happens which the window.
+	 *
+	 * <p>
+	 * On {@code FIRE} the pane is evaluated and results are emitted. The contents of the window
+	 * are kept. {@code FIRE_AND_PURGE} acts like {@code FIRE} but the contents of the pane
+	 * are purged. On {@code CONTINUE} nothing happens, processing continues.
+	 */
+	enum TriggerResult {
 		CONTINUE, FIRE_AND_PURGE, FIRE
 	}
 
-	public interface TriggerContext {
+	/**
+	 * A context object that is given to {@code Trigger} methods to allow them to register timer
+	 * callbacks.
+	 */
+	interface TriggerContext {
+
+		/**
+		 * Register a system time callback. When the current system time passes the specified
+		 * time {@link #onTime(long, TriggerContext)} is called.
+		 *
+		 * @param time The time at which to invoke {@link #onTime(long, TriggerContext)}
+		 */
 		void registerProcessingTimeTimer(long time);
 
+		/**
+		 * Register a watermark callback. When the current watermark passes the specified
+		 * time {@link #onTime(long, TriggerContext)} is called.
+		 *
+		 * @see org.apache.flink.streaming.api.watermark.Watermark
+		 *
+		 * @param time The watermark at which to invoke {@link #onTime(long, TriggerContext)}
+		 */
 		void registerWatermarkTimer(long time);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/WatermarkTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/WatermarkTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/WatermarkTrigger.java
index 6ba8890..5d66ba3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/WatermarkTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/WatermarkTrigger.java
@@ -19,6 +19,12 @@ package org.apache.flink.streaming.api.windowing.triggers;
 
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 
+/**
+ * A {@link Trigger} that fires once the watermark passes the end of the window
+ * to which a pane belongs.
+ *
+ * @see org.apache.flink.streaming.api.watermark.Watermark
+ */
 public class WatermarkTrigger implements Trigger<Object, TimeWindow> {
 	private static final long serialVersionUID = 1L;
 
@@ -50,6 +56,9 @@ public class WatermarkTrigger implements Trigger<Object, TimeWindow> {
 		return "WatermarkTrigger()";
 	}
 
+	/**
+	 * Creates trigger that fires once the watermark passes the end of the window.
+	 */
 	public static WatermarkTrigger create() {
 		return new WatermarkTrigger();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
index 145ad25..017c8ea 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
@@ -33,6 +33,13 @@ import java.util.Map;
 import java.util.Set;
 
 
+/**
+ * An operator that can sort a stream based on timestamps. Arriving elements will be put into
+ * buckets based on their timestamp. Sorting and emission of sorted elements happens once
+ * the watermark passes the end of a bucket.
+ *
+ * @param <T> The type of the elements on which this operator works.
+ */
 public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> implements OneInputStreamOperator<T, T> {
 	private static final long serialVersionUID = 1L;
 
@@ -40,8 +47,13 @@ public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> imple
 
 	private transient Map<Long, List<StreamRecord<T>>> buckets;
 
-	public BucketStreamSortOperator(long granularity) {
-		this.granularity = granularity;
+	/**
+	 * Creates a new sorting operator that creates buckets with the given interval.
+	 *
+	 * @param interval The size (in time) of one bucket.
+	 */
+	public BucketStreamSortOperator(long interval) {
+		this.granularity = interval;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
index ac1a543..50d1cb6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
@@ -19,14 +19,13 @@
 package org.apache.flink.streaming.runtime.operators;
 
 /**
- * This interface must be implemented by objects that are triggered by a
- * {@link TriggerTimer}.
+ * This interface must be implemented by objects that are triggered by the timer service available
+ * to stream operators in {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment}.
  */
 public interface Triggerable {
 
 	/**
-	 * This method is invoked by the {@link TriggerTimer}
-	 * and given the timestamp for which the trigger was scheduled.
+	 * This method is invoked with the timestamp for which the trigger was scheduled.
 	 * <p>
 	 * If the triggering is delayed for whatever reason (trigger timer was blocked, JVM stalled due
 	 * to a garbage collection), the timestamp supplied to this function will still be the original

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
index 53df838..31c7fed 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
@@ -30,7 +30,15 @@ import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuff
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
+/**
+ * Evicting window operator for non-keyed windows.
+ *
+ * @see org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator
+ *
+ * @param <IN> The type of the incoming elements.
+ * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
+ */
 public class EvictingNonKeyedWindowOperator<IN, OUT, W extends Window> extends NonKeyedWindowOperator<IN, OUT, W> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 334eb54..49d58e4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
@@ -33,6 +33,18 @@ import org.slf4j.LoggerFactory;
 
 import java.util.Map;
 
+/**
+ * A {@link WindowOperator} that also allows an {@link Evictor} to be used.
+ *
+ * <p>
+ * The {@code Evictor} is used to evict elements from panes before processing a window and after
+ * a {@link Trigger} has fired.
+ *
+ * @param <K> The type of key returned by the {@code KeySelector}.
+ * @param <IN> The type of the incoming elements.
+ * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
+ */
 public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends WindowOperator<K, IN, OUT, W> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
index d48643d..a80242d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
@@ -45,6 +45,15 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * Window operator for non-keyed windows.
+ *
+ * @see org.apache.flink.streaming.runtime.operators.windowing.WindowOperator
+ *
+ * @param <IN> The type of the incoming elements.
+ * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
+ */
 public class NonKeyedWindowOperator<IN, OUT, W extends Window>
 		extends AbstractUdfStreamOperator<OUT, AllWindowFunction<IN, OUT, W>>
 		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 2d4635f..548afb3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -46,6 +46,32 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * An operator that implements the logic for windowing based on a {@link WindowAssigner} and
+ * {@link Trigger}.
+ *
+ * <p>
+ * When an element arrives it gets assigned a key using a {@link KeySelector} and it get's
+ * assigned to zero or more windows using a {@link WindowAssigner}. Based on this the element
+ * is put into panes. A pane is the bucket of elements that have the same key and same
+ * {@code Window}. An element can be in multiple panes of it was assigned to multiple windows by the
+ * {@code WindowAssigner}.
+ *
+ * <p>
+ * Each pane gets its own instance of the provided {@code Trigger}. This trigger determines when
+ * the contents of the pane should be processed to emit results. When a trigger fires,
+ * the given {@link WindowFunction} is invoked to produce the results that are emitted for
+ * the pane to which the {@code Trigger} belongs.
+ *
+ * <p>
+ * This operator also needs a {@link WindowBufferFactory} to create a buffer for storing the
+ * elements of each pane.
+ *
+ * @param <K> The type of key returned by the {@code KeySelector}.
+ * @param <IN> The type of the incoming elements.
+ * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
+ */
 public class WindowOperator<K, IN, OUT, W extends Window>
 		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, K, W>>
 		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
@@ -54,24 +80,47 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 
 	private static final Logger LOG = LoggerFactory.getLogger(WindowOperator.class);
 
-
 	private final WindowAssigner<? super IN, W> windowAssigner;
+
 	private final KeySelector<IN, K> keySelector;
 
 	private final Trigger<? super IN, ? super W> triggerTemplate;
+
 	private final WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory;
 
+	/**
+	 * The windows (panes) that are currently in-flight. Each pane has a {@code WindowBuffer}
+	 * and a {@code TriggerContext} that stores the {@code Trigger} for that pane.
+	 */
 	protected transient Map<K, Map<W, Tuple2<WindowBuffer<IN>, TriggerContext>>> windows;
 
+	/**
+	 * Processing time timers that are currently in-flight.
+	 */
 	private transient Map<Long, Set<TriggerContext>> processingTimeTimers;
+
+	/**
+	 * Current waiting watermark callbacks.
+	 */
 	private transient Map<Long, Set<TriggerContext>> watermarkTimers;
 
+	/**
+	 * This is given to the {@code WindowFunction} for emitting elements with a given timestamp.
+	 */
 	protected transient TimestampedCollector<OUT> timestampedCollector;
 
+	/**
+	 * If this is true. The current processing time is set as the timestamp of incoming elements.
+	 * This for use with a {@link org.apache.flink.streaming.api.windowing.evictors.TimeEvictor}
+	 * if eviction should happen based on processing time.
+	 */
 	private boolean setProcessingTime = false;
 
 	private TypeSerializer<IN> inputSerializer;
 
+	/**
+	 * Creates a new {@code WindowOperator} based on the given policies and user functions.
+	 */
 	public WindowOperator(WindowAssigner<? super IN, W> windowAssigner,
 			KeySelector<IN, K> keySelector,
 			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
@@ -245,6 +294,10 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 		}
 	}
 
+	/**
+	 * A context object that is given to {@code Trigger} functions to allow them to register
+	 * timer/watermark callbacks.
+	 */
 	protected class TriggerContext implements Trigger.TriggerContext {
 		Trigger<? super IN, ? super W> trigger;
 		K key;
@@ -312,9 +365,4 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 	public WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> getWindowBufferFactory() {
 		return windowBufferFactory;
 	}
-
-	@VisibleForTesting
-	public boolean isSetProcessingTime() {
-		return setProcessingTime;
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
index 50e392b..28365e1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
@@ -17,6 +17,19 @@
  */
 package org.apache.flink.streaming.runtime.operators.windowing.buffers;
 
+/**
+ * A {@code WindowBuffer} that can also evict elements from the buffer. The order in which
+ * the elements are added is preserved. Elements can only be evicted started from the beginning of
+ * the buffer.
+ *
+ * @param <T> The type of elements that this {@code WindowBuffer} can store.
+ */
+
 public interface EvictingWindowBuffer<T> extends WindowBuffer<T> {
-	public boolean removeElements(int count);
+
+	/**
+	 * Removes the given number of elements, starting from the beginning.
+	 * @param count The number of elements to remove.
+	 */
+	void removeElements(int count);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
index 092718a..f9f8b26 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
@@ -25,6 +25,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import java.util.ArrayDeque;
 
+/**
+ * An {@link EvictingWindowBuffer} that stores elements on the Java Heap.
+ *
+ * @param <T> The type of elements that this {@code WindowBuffer} can store.
+ */
 public class HeapWindowBuffer<T> implements EvictingWindowBuffer<T> {
 	private static final long serialVersionUID = 1L;
 
@@ -40,12 +45,11 @@ public class HeapWindowBuffer<T> implements EvictingWindowBuffer<T> {
 	}
 
 	@Override
-	public boolean removeElements(int count) {
+	public void removeElements(int count) {
 		// TODO determine if this can be done in a better way
 		for (int i = 0; i < count; i++) {
 			elements.removeFirst();
 		}
-		return false;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
index 85f90b0..37be8f0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
@@ -25,6 +25,13 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import java.util.Collections;
 
+/**
+ * An {@link WindowBuffer} that stores elements on the Java Heap. This buffer uses a
+ * {@link ReduceFunction} to pre-aggregate elements that are added to the buffer.
+ *
+ * @param <T> The type of elements that this {@code WindowBuffer} can store.
+ */
+
 public class PreAggregatingHeapWindowBuffer<T> implements WindowBuffer<T> {
 	private static final long serialVersionUID = 1L;
 
@@ -85,7 +92,7 @@ public class PreAggregatingHeapWindowBuffer<T> implements WindowBuffer<T> {
 
 		@Override
 		public PreAggregatingHeapWindowBuffer<T> create() {
-			return new PreAggregatingHeapWindowBuffer<T>(reduceFunction);
+			return new PreAggregatingHeapWindowBuffer<>(reduceFunction);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
index 8c891d5..b111667 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
@@ -18,17 +18,47 @@
 package org.apache.flink.streaming.runtime.operators.windowing.buffers;
 
 
+import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import java.io.Serializable;
 
+/**
+ * A {@code WindowBuffer} is used by
+ * {@link org.apache.flink.streaming.runtime.operators.windowing.WindowOperator} to store
+ * the elements of one pane.
+ *
+ * <p>
+ * A pane is the bucket of elements that have the same key (assigned by the
+ * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
+ * be in multiple panes of it was assigned to multiple windows by the
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
+ * have their own instance of the {@code Evictor}.
+ *
+ * @param <T> The type of elements that this {@code WindowBuffer} can store.
+ */
 public interface WindowBuffer<T> extends Serializable {
 
-	public void storeElement(StreamRecord<T> element) throws Exception;
+	/**
+	 * Adds the element to the buffer.
+	 *
+	 * @param element The element to add.
+	 */
+	void storeElement(StreamRecord<T> element) throws Exception;
 
-	public Iterable<StreamRecord<T>> getElements();
+	/**
+	 * Returns all elements that are currently in the buffer.
+	 */
+	Iterable<StreamRecord<T>> getElements();
 
-	public Iterable<T> getUnpackedElements();
+	/**
+	 * Returns all elements that are currently in the buffer. This will unwrap the contained
+	 * elements from their {@link StreamRecord}.
+	 */
+	Iterable<T> getUnpackedElements();
 
-	public int size();
+	/**
+	 * Returns the number of elements that are currently in the buffer.
+	 */
+	int size();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/62df0a03/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
index 4a7f6df..4bcdf09 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
@@ -22,9 +22,33 @@ import org.apache.flink.configuration.Configuration;
 
 import java.io.Serializable;
 
+/**
+ * A factory for {@link WindowBuffer WindowBuffers}.
+ *
+ * @param <T> The type of elements that the created {@code WindowBuffer} can store.
+ * @param <B> The type of the created {@code WindowBuffer}
+ */
 public interface WindowBufferFactory<T, B extends WindowBuffer<T>> extends Serializable {
+
+	/**
+	 * Sets the {@link RuntimeContext} that is used to initialize eventual user functions
+	 * inside the created buffers.
+	 */
 	void setRuntimeContext(RuntimeContext ctx);
+
+	/**
+	 * Calls {@code open()} on eventual user functions inside the buffer.
+	 */
 	void open(Configuration config) throws Exception;
+
+	/**
+	 * Calls {@code close()} on eventual user functions inside the buffer.
+	 */
+
 	void close() throws Exception;
+
+	/**
+	 * Creates a new {@code WindowBuffer}.
+	 */
 	B create();
 }