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/07 22:33:50 UTC

[1/8] flink git commit: [FLINK-2674] Add Fold Window Operation for new Windowing API

Repository: flink
Updated Branches:
  refs/heads/master 47b5cb795 -> f9eea5e5a


[FLINK-2674] Add Fold Window Operation for new Windowing API


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

Branch: refs/heads/master
Commit: f9eea5e5a7d13d844e06d3eb6849268b3eaf4c08
Parents: ce792b1
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Oct 7 16:49:40 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       |  34 ++++
 .../api/datastream/WindowedStream.java          |  34 ++++
 .../windowing/FoldAllWindowFunction.java        |  97 ++++++++++
 .../functions/windowing/FoldWindowFunction.java |  97 ++++++++++
 .../windowing/NonKeyedWindowOperator.java       |  12 +-
 .../operators/windowing/WindowOperator.java     |  12 +-
 .../operators/windowing/WindowFoldITCase.java   | 191 +++++++++++++++++++
 .../streaming/api/scala/AllWindowedStream.scala |  43 ++++-
 .../streaming/api/scala/WindowedStream.scala    |  43 ++++-
 .../streaming/api/scala/WindowFoldITCase.scala  | 148 ++++++++++++++
 10 files changed, 707 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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 a8d7654..c7a70d7 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
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.api.datastream;
 
 import org.apache.commons.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -29,6 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 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.windowing.FoldAllWindowFunction;
 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;
@@ -164,6 +166,38 @@ public class AllWindowedStream<T, W extends Window> {
 	}
 
 	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+				Utils.getCallLocationName(), true);
+
+		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
 	 * 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.

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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 99f7d06..42e0bd7 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
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.api.datastream;
 
 import org.apache.commons.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -30,6 +31,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 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.windowing.FoldWindowFunction;
 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;
@@ -178,6 +180,38 @@ public class WindowedStream<T, K, W extends Window> {
 	}
 
 	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+				Utils.getCallLocationName(), true);
+
+		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, 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.

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
new file mode 100644
index 0000000..69f24fe
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.functions.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class FoldAllWindowFunction<W extends Window, T, R>
+		extends WrappingFunction<FoldFunction<T, R>>
+		implements AllWindowFunction<T, R, W>, OutputTypeConfigurable<R> {
+	private static final long serialVersionUID = 1L;
+
+	private byte[] serializedInitialValue;
+	private TypeSerializer<R> outSerializer;
+	private transient R initialValue;
+
+	public FoldAllWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
+		super(reduceFunction);
+		this.initialValue = initialValue;
+	}
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		super.open(configuration);
+
+		if (serializedInitialValue == null) {
+			throw new RuntimeException("No initial value was serialized for the fold " +
+					"window function. Probably the setOutputType method was not called.");
+		}
+
+		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
+		InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
+				new DataInputStream(bais)
+		);
+		initialValue = outSerializer.deserialize(in);
+	}
+
+	@Override
+	public void apply(W window, Iterable<T> values, Collector<R> out) throws Exception {
+		R result = outSerializer.copy(initialValue);
+
+		for (T val: values) {
+			result = wrappedFunction.fold(result, val);
+		}
+
+		out.collect(result);
+	}
+
+	@Override
+	public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
+		outSerializer = outTypeInfo.createSerializer(executionConfig);
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
+				new DataOutputStream(baos)
+		);
+
+		try {
+			outSerializer.serialize(initialValue, out);
+		} catch (IOException ioe) {
+			throw new RuntimeException("Unable to serialize initial value of type " +
+					initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
+		}
+
+		serializedInitialValue = baos.toByteArray();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
new file mode 100644
index 0000000..04d2ac7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.functions.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class FoldWindowFunction<K, W extends Window, T, R>
+		extends WrappingFunction<FoldFunction<T, R>>
+		implements WindowFunction<T, R, K, W>, OutputTypeConfigurable<R> {
+	private static final long serialVersionUID = 1L;
+
+	private byte[] serializedInitialValue;
+	private TypeSerializer<R> outSerializer;
+	private transient R initialValue;
+
+	public FoldWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
+		super(reduceFunction);
+		this.initialValue = initialValue;
+	}
+
+	@Override
+	public void open(Configuration configuration) throws Exception {
+		super.open(configuration);
+
+		if (serializedInitialValue == null) {
+			throw new RuntimeException("No initial value was serialized for the fold " +
+					"window function. Probably the setOutputType method was not called.");
+		}
+
+		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
+		InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
+				new DataInputStream(bais)
+		);
+		initialValue = outSerializer.deserialize(in);
+	}
+
+	@Override
+	public void apply(K k, W window, Iterable<T> values, Collector<R> out) throws Exception {
+		R result = outSerializer.copy(initialValue);
+
+		for (T val: values) {
+			result = wrappedFunction.fold(result, val);
+		}
+
+		out.collect(result);
+	}
+
+	@Override
+	public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
+		outSerializer = outTypeInfo.createSerializer(executionConfig);
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
+				new DataOutputStream(baos)
+		);
+
+		try {
+			outSerializer.serialize(initialValue, out);
+		} catch (IOException ioe) {
+			throw new RuntimeException("Unable to serialize initial value of type " +
+					initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
+		}
+
+		serializedInitialValue = baos.toByteArray();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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 a80242d..e6aa53b 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
@@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration;
 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.OutputTypeConfigurable;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -56,7 +57,7 @@ import java.util.Set;
  */
 public class NonKeyedWindowOperator<IN, OUT, W extends Window>
 		extends AbstractUdfStreamOperator<OUT, AllWindowFunction<IN, OUT, W>>
-		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -268,6 +269,15 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
 		return this;
 	}
 
+	@Override
+	public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
+		if (userFunction instanceof OutputTypeConfigurable) {
+			@SuppressWarnings("unchecked")
+			OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
+			typeConfigurable.setOutputType(outTypeInfo, executionConfig);
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	// Getters for testing
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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 368b8fa..7762101 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
@@ -30,6 +30,7 @@ import org.apache.flink.configuration.Configuration;
 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.OutputTypeConfigurable;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -74,7 +75,7 @@ import java.util.Set;
  */
 public class WindowOperator<K, IN, OUT, W extends Window>
 		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, K, W>>
-		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -342,6 +343,15 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 		return this;
 	}
 
+	@Override
+	public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
+		if (userFunction instanceof OutputTypeConfigurable) {
+			@SuppressWarnings("unchecked")
+			OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
+			typeConfigurable.setOutputType(outTypeInfo, executionConfig);
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	// Getters for testing
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
new file mode 100644
index 0000000..45649bd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
@@ -0,0 +1,191 @@
+/*
+* 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 com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
+ * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
+ */
+public class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
+
+	private static List<String> testResults;
+
+	@Test
+	public void testFoldWindow() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+				ctx.collect(Tuple2.of("a", 2));
+
+				ctx.collect(Tuple2.of("b", 3));
+				ctx.collect(Tuple2.of("b", 4));
+				ctx.collect(Tuple2.of("b", 5));
+
+				ctx.collect(Tuple2.of("a", 6));
+				ctx.collect(Tuple2.of("a", 7));
+				ctx.collect(Tuple2.of("a", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+		source1
+				.keyBy(0)
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+					@Override
+					public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
+							Tuple2<String, Integer> value) throws Exception {
+						accumulator.f0 += value.f0;
+						accumulator.f1 += value.f1;
+						return accumulator;
+					}
+				})
+				.addSink(new SinkFunction<Tuple2<String, Integer>>() {
+					@Override
+					public void invoke(Tuple2<String, Integer> value) throws Exception {
+						testResults.add(value.toString());
+					}
+				});
+
+		env.execute("Fold Window Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(R:aaa,3)",
+				"(R:aaa,21)",
+				"(R:bbb,12)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	@Test
+	public void testFoldAllWindow() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+				ctx.collect(Tuple2.of("a", 2));
+
+				ctx.collect(Tuple2.of("b", 3));
+				ctx.collect(Tuple2.of("a", 3));
+				ctx.collect(Tuple2.of("b", 4));
+				ctx.collect(Tuple2.of("a", 4));
+				ctx.collect(Tuple2.of("b", 5));
+				ctx.collect(Tuple2.of("a", 5));
+
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+		source1
+				.windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+					@Override
+					public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
+							Tuple2<String, Integer> value) throws Exception {
+						accumulator.f0 += value.f0;
+						accumulator.f1 += value.f1;
+						return accumulator;
+					}
+				})
+				.addSink(new SinkFunction<Tuple2<String, Integer>>() {
+					@Override
+					public void invoke(Tuple2<String, Integer> value) throws Exception {
+						testResults.add(value.toString());
+					}
+				});
+
+		env.execute("Fold All-Window Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(R:aaa,3)",
+				"(R:bababa,24)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1;
+		}
+
+		@Override
+		public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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
index d2d0a1d..65cafb7 100644
--- 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
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.api.scala
 
-import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.functions.{FoldFunction, 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.aggregation.AggregationFunction.AggregationType
@@ -122,6 +122,47 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
   }
 
   /**
+   * Applies the given fold function to each 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.
+   *
+   * @param function The fold function.
+   * @return The data stream that is the result of applying the fold function to the window.
+   */
+  def fold[R: TypeInformation: ClassTag](
+      initialValue: R,
+      function: FoldFunction[T,R]): DataStream[R] = {
+    if (function == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+
+    val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
+
+    javaStream.fold(initialValue, function, resultType)
+  }
+
+  /**
+   * Applies the given fold function to each 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.
+   *
+   * @param function The fold function.
+   * @return The data stream that is the result of applying the fold function to the window.
+   */
+  def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
+    if (function == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+    val cleanFun = clean(function)
+    val folder = new FoldFunction[T,R] {
+      def fold(acc: R, v: T) = {
+        cleanFun(acc, v)
+      }
+    }
+    fold(initialValue, folder)
+  }
+
+  /**
    * 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.

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/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
index 3963765..a8ddaf8 100644
--- 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
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.api.scala
 
-import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.functions.{FoldFunction, 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.aggregation.AggregationFunction.AggregationType
@@ -125,6 +125,47 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
   }
 
   /**
+   * Applies the given fold function to each 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.
+   *
+   * @param function The fold function.
+   * @return The data stream that is the result of applying the fold function to the window.
+   */
+  def fold[R: TypeInformation: ClassTag](
+      initialValue: R,
+      function: FoldFunction[T,R]): DataStream[R] = {
+    if (function == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+
+    val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
+
+    javaStream.fold(initialValue, function, resultType)
+  }
+
+  /**
+   * Applies the given fold function to each 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.
+   *
+   * @param function The fold function.
+   * @return The data stream that is the result of applying the fold function to the window.
+   */
+  def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
+    if (function == null) {
+      throw new NullPointerException("Fold function must not be null.")
+    }
+    val cleanFun = clean(function)
+    val folder = new FoldFunction[T,R] {
+      def fold(acc: R, v: T) = {
+        cleanFun(acc, v)
+      }
+    }
+    fold(initialValue, folder)
+  }
+
+  /**
    * 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.

http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
new file mode 100644
index 0000000..dd098a0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.scala
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.TimestampExtractor
+import org.apache.flink.streaming.api.functions.sink.SinkFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.junit.Test
+import org.junit.Assert._
+
+import scala.collection.mutable
+
+/**
+ * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
+ * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
+ */
+class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testFoldWindow(): Unit = {
+    WindowFoldITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setParallelism(1)
+
+    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+        ctx.collect(("a", 0))
+        ctx.collect(("a", 1))
+        ctx.collect(("a", 2))
+        ctx.collect(("b", 3))
+        ctx.collect(("b", 4))
+        ctx.collect(("b", 5))
+        ctx.collect(("a", 6))
+        ctx.collect(("a", 7))
+        ctx.collect(("a", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new WindowFoldITCase.Tuple2TimestampExtractor)
+
+    source1
+      .keyBy(0)
+      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+      .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
+      .addSink(new SinkFunction[(String, Int)]() {
+        def invoke(value: (String, Int)) {
+        WindowFoldITCase.testResults += value.toString
+        }
+      })
+
+    env.execute("Fold Window Test")
+
+    val expectedResult = mutable.MutableList(
+      "(R:aaa,3)",
+      "(R:aaa,21)",
+      "(R:bbb,12)")
+
+    assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
+  }
+
+  @Test
+  def testFoldAllWindow(): Unit = {
+    WindowFoldITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setParallelism(1)
+
+    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+        ctx.collect(("a", 0))
+        ctx.collect(("a", 1))
+        ctx.collect(("a", 2))
+        ctx.collect(("b", 3))
+        ctx.collect(("a", 3))
+        ctx.collect(("b", 4))
+        ctx.collect(("a", 4))
+        ctx.collect(("b", 5))
+        ctx.collect(("a", 5))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new WindowFoldITCase.Tuple2TimestampExtractor)
+
+    source1
+      .windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+      .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
+      .addSink(new SinkFunction[(String, Int)]() {
+      def invoke(value: (String, Int)) {
+        WindowFoldITCase.testResults += value.toString
+      }
+    })
+
+    env.execute("Fold All-Window Test")
+
+    val expectedResult = mutable.MutableList(
+      "(R:aaa,3)",
+      "(R:bababa,24)")
+
+    assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
+  }
+
+}
+
+
+object WindowFoldITCase {
+  private var testResults: mutable.MutableList[String] = null
+
+  private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
+    def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
+      element._2
+    }
+
+    def emitWatermark(element: (String, Int), currentTimestamp: Long): Long = {
+      element._2 - 1
+    }
+
+    def getCurrentWatermark: Long = {
+      Long.MinValue
+    }
+  }
+}


[5/8] flink git commit: [FLINK-2819] Add Windowed Join/CoGroup Operator Based on Tagged Union

Posted by al...@apache.org.
[FLINK-2819] Add Windowed Join/CoGroup Operator Based on Tagged Union

Right now, this does everything in memory, so the JVM will blow if data
for one key becomes too large.


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

Branch: refs/heads/master
Commit: 8634dbbe998af53471bed2f3a6557c722bb37b87
Parents: 47b5cb7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 16:33:04 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/CoGroupedStreams.java        | 563 +++++++++++++++++++
 .../streaming/api/datastream/DataStream.java    |  36 +-
 .../streaming/api/datastream/JoinedStreams.java | 328 +++++++++++
 .../datastream/temporal/StreamJoinOperator.java | 274 ---------
 .../datastream/temporal/TemporalOperator.java   | 124 ----
 .../api/datastream/temporal/TemporalWindow.java |  45 --
 .../windowing/ReduceWindowFunction.java         |  32 +-
 .../flink/streaming/api/CoGroupJoinITCase.java  | 372 ++++++++++++
 .../streaming/api/WindowCrossJoinTest.java      | 143 -----
 .../api/operators/co/SelfConnectionTest.java    |  61 --
 .../streaming/examples/join/WindowJoin.java     | 128 +++--
 .../examples/join/util/WindowJoinData.java      |  70 +--
 .../scala/examples/join/WindowJoin.scala        |  43 +-
 .../join/WindowJoinITCase.java                  | 101 ++--
 .../join/WindowJoinITCase.java                  | 101 ++--
 .../streaming/api/scala/CoGroupedStreams.scala  | 294 ++++++++++
 .../flink/streaming/api/scala/DataStream.scala  |  24 +-
 .../streaming/api/scala/JoinedStreams.scala     | 303 ++++++++++
 .../api/scala/StreamJoinOperator.scala          | 203 -------
 .../streaming/api/scala/TemporalOperator.scala  |  51 --
 .../streaming/api/scala/CoGroupJoinITCase.scala | 274 +++++++++
 .../StreamingScalaAPICompletenessTest.scala     |  14 +-
 22 files changed, 2409 insertions(+), 1175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
new file mode 100644
index 0000000..e1f1a96
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -0,0 +1,563 @@
+/*
+ * 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.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+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.util.Collector;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ *{@code CoGroupedStreams} represents two {@link DataStream DataStreams} that have been co-grouped.
+ * A streaming co-group operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize co-group operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> two = ...;
+ *
+ * DataStream<T> result = one.coGroup(two)
+ *     .where(new MyFirstKeySelector())
+ *     .equalTo(new MyFirstKeySelector())
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyCoGroupFunction());
+ * } </pre>
+ */
+public class CoGroupedStreams {
+
+	/**
+	 * A co-group operation that does not yet have its {@link KeySelector KeySelectors} defined.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 */
+	public static class Unspecified<T1, T2> {
+		DataStream<T1> input1;
+		DataStream<T2> input2;
+
+		protected Unspecified(DataStream<T1> input1,
+				DataStream<T2> input2) {
+			this.input1 = input1;
+			this.input2 = input2;
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the first input.
+		 */
+		public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector)  {
+			return new WithKey<>(input1, input2, input1.clean(keySelector), null);
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector)  {
+			return new WithKey<>(input1, input2, null, input1.clean(keySelector));
+		}
+	}
+
+	/**
+	 * A co-group operation that has {@link KeySelector KeySelectors} defined for either both or
+	 * one input.
+	 *
+	 * <p>
+	 * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
+	 * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
+	 * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 */
+	public static class WithKey<T1, T2, KEY> {
+		DataStream<T1> input1;
+		DataStream<T2> input2;
+
+		KeySelector<T1, KEY> keySelector1;
+		KeySelector<T2, KEY> keySelector2;
+
+		protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
+			this.input1 = input1;
+			this.input2 = input2;
+
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the first input.
+		 */
+		public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector)  {
+			return new CoGroupedStreams.WithKey<>(input1, input2, input1.clean(keySelector), keySelector2);
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public CoGroupedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector)  {
+			return new CoGroupedStreams.WithKey<>(input1, input2, keySelector1, input1.clean(keySelector));
+		}
+
+		/**
+		 * Specifies the window on which the co-group operation works.
+		 */
+		public <W extends Window> CoGroupedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+			if (keySelector1 == null || keySelector2 == null) {
+				throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+
+			}
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
+		}
+	}
+
+	/**
+	 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs as
+	 * well as a {@link WindowAssigner}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 * @param <W> Type of {@link Window} on which the co-group operation works.
+	 */
+	public static class WithWindow<T1, T2, KEY, W extends Window> {
+		private final DataStream<T1> input1;
+		private final DataStream<T2> input2;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+
+		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+		protected WithWindow(DataStream<T1> input1,
+				DataStream<T2> input2,
+				KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2,
+				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+			this.input1 = input1;
+			this.input2 = input2;
+
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+
+			this.windowAssigner = windowAssigner;
+			this.trigger = trigger;
+			this.evictor = evictor;
+		}
+
+		/**
+		 * Sets the {@code Trigger} that should be used to trigger window emission.
+		 */
+		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+		}
+
+		/**
+		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+		}
+
+		/**
+		 * Completes the co-group operation with the user function that is executed
+		 * for windowed groups.
+		 */
+		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
+
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					CoGroupFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"CoGroup",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the co-group operation with the user function that is executed
+		 * for windowed groups.
+		 */
+		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
+					.map(new Input1Tagger<T1, T2>())
+					.returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+			DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
+					.map(new Input2Tagger<T1, T2>())
+					.returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+
+			WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = taggedInput1
+					.union(taggedInput2)
+					.keyBy(new UnionKeySelector<>(keySelector1, keySelector2))
+					.window(windowAssigner);
+
+			if (trigger != null) {
+				windowOp.trigger(trigger);
+			}
+			if (evictor != null) {
+				windowOp.evictor(evictor);
+			}
+
+			return windowOp.apply(new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType);
+		}
+	}
+
+	/**
+	 * Creates a new co-group operation from the two given inputs.
+	 */
+	public static <T1, T2> Unspecified<T1, T2> createCoGroup(DataStream<T1> input1, DataStream<T2> input2) {
+		return new Unspecified<>(input1, input2);
+	}
+
+	/**
+	 * Internal class for implementing tagged union co-group.
+	 */
+	public static class TaggedUnion<T1, T2> {
+		private final T1 one;
+		private final T2 two;
+
+		private TaggedUnion(T1 one, T2 two) {
+			this.one = one;
+			this.two = two;
+		}
+
+		public boolean isOne() {
+			return one != null;
+		}
+
+		public boolean isTwo() {
+			return two != null;
+		}
+
+		public T1 getOne() {
+			return one;
+		}
+
+		public T2 getTwo() {
+			return two;
+		}
+
+		public static <T1, T2> TaggedUnion<T1, T2> one(T1 one) {
+			return new TaggedUnion<>(one, null);
+		}
+
+		public static <T1, T2> TaggedUnion<T1, T2> two(T2 two) {
+			return new TaggedUnion<>(null, two);
+		}
+	}
+
+	private static class UnionTypeInfo<T1, T2> extends TypeInformation<TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		TypeInformation<T1> oneType;
+		TypeInformation<T2> twoType;
+
+		public UnionTypeInfo(TypeInformation<T1> oneType,
+				TypeInformation<T2> twoType) {
+			this.oneType = oneType;
+			this.twoType = twoType;
+		}
+
+		@Override
+		public boolean isBasicType() {
+			return false;
+		}
+
+		@Override
+		public boolean isTupleType() {
+			return false;
+		}
+
+		@Override
+		public int getArity() {
+			return 2;
+		}
+
+		@Override
+		public int getTotalFields() {
+			return 2;
+		}
+
+		@Override
+		@SuppressWarnings("unchecked, rawtypes")
+		public Class<TaggedUnion<T1, T2>> getTypeClass() {
+			return (Class) TaggedUnion.class;
+		}
+
+		@Override
+		public boolean isKeyType() {
+			return true;
+		}
+
+		@Override
+		public TypeSerializer<TaggedUnion<T1, T2>> createSerializer(ExecutionConfig config) {
+			return new UnionSerializer<>(oneType.createSerializer(config), twoType.createSerializer(config));
+		}
+
+		@Override
+		public String toString() {
+			return "TaggedUnion<" + oneType + ", " + twoType + ">";
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj instanceof UnionTypeInfo) {
+				@SuppressWarnings("unchecked")
+				UnionTypeInfo<T1, T2> unionTypeInfo = (UnionTypeInfo<T1, T2>) obj;
+
+				return unionTypeInfo.canEqual(this) && oneType.equals(unionTypeInfo.oneType) && twoType.equals(unionTypeInfo.twoType);
+			} else {
+				return false;
+			}
+		}
+
+		@Override
+		public int hashCode() {
+			return 31 *  oneType.hashCode() + twoType.hashCode();
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof UnionTypeInfo;
+		}
+	}
+
+	private static class UnionSerializer<T1, T2> extends TypeSerializer<TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		private final TypeSerializer<T1> oneSerializer;
+		private final TypeSerializer<T2> twoSerializer;
+
+		public UnionSerializer(TypeSerializer<T1> oneSerializer,
+				TypeSerializer<T2> twoSerializer) {
+			this.oneSerializer = oneSerializer;
+			this.twoSerializer = twoSerializer;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return false;
+		}
+
+		@Override
+		public TypeSerializer<TaggedUnion<T1, T2>> duplicate() {
+			return this;
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> createInstance() {
+			return null;
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from) {
+			if (from.isOne()) {
+				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+			} else {
+				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from, TaggedUnion<T1, T2> reuse) {
+			if (from.isOne()) {
+				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+			} else {
+				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+			}		}
+
+		@Override
+		public int getLength() {
+			return 0;
+		}
+
+		@Override
+		public void serialize(TaggedUnion<T1, T2> record, DataOutputView target) throws IOException {
+			if (record.isOne()) {
+				target.writeByte(1);
+				oneSerializer.serialize(record.getOne(), target);
+			} else {
+				target.writeByte(2);
+				twoSerializer.serialize(record.getTwo(), target);
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> deserialize(DataInputView source) throws IOException {
+			byte tag = source.readByte();
+			if (tag == 1) {
+				return TaggedUnion.one(oneSerializer.deserialize(source));
+			} else {
+				return TaggedUnion.two(twoSerializer.deserialize(source));
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> deserialize(TaggedUnion<T1, T2> reuse,
+				DataInputView source) throws IOException {
+			byte tag = source.readByte();
+			if (tag == 1) {
+				return TaggedUnion.one(oneSerializer.deserialize(source));
+			} else {
+				return TaggedUnion.two(twoSerializer.deserialize(source));
+			}
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			byte tag = source.readByte();
+			target.writeByte(tag);
+			if (tag == 1) {
+				oneSerializer.copy(source, target);
+			} else {
+				twoSerializer.copy(source, target);
+			}
+		}
+
+		@Override
+		public int hashCode() {
+			return 31 * oneSerializer.hashCode() + twoSerializer.hashCode();
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public boolean equals(Object obj) {
+			if (obj instanceof UnionSerializer) {
+				UnionSerializer<T1, T2> other = (UnionSerializer<T1, T2>) obj;
+
+				return other.canEqual(this) && oneSerializer.equals(other.oneSerializer) && twoSerializer.equals(other.twoSerializer);
+			} else {
+				return false;
+			}
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof UnionSerializer;
+		}
+	}
+
+	private static class Input1Tagger<T1, T2> implements MapFunction<T1, TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public TaggedUnion<T1, T2> map(T1 value) throws Exception {
+			return TaggedUnion.one(value);
+		}
+	}
+
+	private static class Input2Tagger<T1, T2> implements MapFunction<T2, TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public TaggedUnion<T1, T2> map(T2 value) throws Exception {
+			return TaggedUnion.two(value);
+		}
+	}
+
+	private static class UnionKeySelector<T1, T2, KEY> implements KeySelector<TaggedUnion<T1, T2>, KEY> {
+		private static final long serialVersionUID = 1L;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+
+		public UnionKeySelector(KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2) {
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+		}
+
+		@Override
+		public KEY getKey(TaggedUnion<T1, T2> value) throws Exception{
+			if (value.isOne()) {
+				return keySelector1.getKey(value.getOne());
+			} else {
+				return keySelector2.getKey(value.getTwo());
+			}
+		}
+	}
+
+	private static class CoGroupWindowFunction<T1, T2, T, KEY, W extends Window>
+			extends WrappingFunction<CoGroupFunction<T1, T2, T>>
+			implements WindowFunction<TaggedUnion<T1, T2>, T, KEY, W> {
+		private static final long serialVersionUID = 1L;
+
+		public CoGroupWindowFunction(CoGroupFunction<T1, T2, T> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void apply(KEY key,
+				W window,
+				Iterable<TaggedUnion<T1, T2>> values,
+				Collector<T> out) throws Exception {
+			List<T1> oneValues = Lists.newArrayList();
+			List<T2> twoValues = Lists.newArrayList();
+			for (TaggedUnion<T1, T2> val: values) {
+				if (val.isOne()) {
+					oneValues.add(val.getOne());
+				} else {
+					twoValues.add(val.getTwo());
+				}
+			}
+			wrappedFunction.coGroup(oneValues, twoValues, out);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 8de1a0d..0be1d56 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.StreamJoinOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.TimestampExtractor;
 import org.apache.flink.streaming.api.functions.sink.FileSinkFunctionByMillis;
@@ -618,30 +617,19 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Initiates a temporal Join transformation. <br/>
-	 * A temporal Join transformation joins the elements of two
-	 * {@link DataStream}s on key equality over a specified time window.
-	 *
-	 * <p>
-	 * This method returns a {@link StreamJoinOperator} on which the
-	 * {@link StreamJoinOperator#onWindow(long, java.util.concurrent.TimeUnit)}
-	 * should be called to define the window, and then the
-	 * {@link StreamJoinOperator.JoinWindow#where(int...)} and
-	 * {@link StreamJoinOperator.JoinPredicate#equalTo(int...)} can be used to define
-	 * the join keys.
-	 * <p>
-	 * The user can also use the
-	 * {@link org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator.JoinPredicate.JoinedStream#with}
-	 * to apply a custom join function.
-	 * 
-	 * @param dataStreamToJoin
-	 *            The other DataStream with which this DataStream is joined.
-	 * @return A {@link StreamJoinOperator} to continue the definition of the
-	 *         Join transformation.
-	 * 
+	 * Creates a join operation. See {@link CoGroupedStreams} for an example of how the keys
+	 * and window can be specified.
+	 */
+	public <T2> CoGroupedStreams.Unspecified<T, T2> coGroup(DataStream<T2> otherStream) {
+		return CoGroupedStreams.createCoGroup(this, otherStream);
+	}
+
+	/**
+	 * Creates a join operation. See {@link JoinedStreams} for an example of how the keys
+	 * and window can be specified.
 	 */
-	public <IN2> StreamJoinOperator<T, IN2> join(DataStream<IN2> dataStreamToJoin) {
-		return new StreamJoinOperator<T, IN2>(this, dataStreamToJoin);
+	public <T2> JoinedStreams.Unspecified<T, T2> join(DataStream<T2> otherStream) {
+		return JoinedStreams.createJoin(this, otherStream);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
new file mode 100644
index 0000000..ee848e3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
@@ -0,0 +1,328 @@
+/*
+ * 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.CoGroupFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+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.util.Collector;
+
+/**
+ *{@code JoinedStreams} represents two {@link DataStream DataStreams} that have been joined.
+ * A streaming join operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize the join operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the the join is being evaluated in memory so you need to ensure that the number
+ * of elements per key does not get too high. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> twp = ...;
+ *
+ * DataStream<T> result = one.join(two)
+ *     .where(new MyFirstKeySelector())
+ *     .equalTo(new MyFirstKeySelector())
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyJoinFunction());
+ * } </pre>
+ */
+public class JoinedStreams extends CoGroupedStreams{
+
+	/**
+	 * A join operation that does not yet have its {@link KeySelector KeySelectors} defined.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 */
+	public static class Unspecified<T1, T2> {
+		DataStream<T1> input1;
+		DataStream<T2> input2;
+
+		protected Unspecified(DataStream<T1> input1,
+				DataStream<T2> input2) {
+			this.input1 = input1;
+			this.input2 = input2;
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the first input.
+		 */
+		public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector)  {
+			return new WithKey<>(input1, input2, keySelector, null);
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector)  {
+			return new WithKey<>(input1, input2, null, keySelector);
+		}
+	}
+
+	/**
+	 * A join operation that has {@link KeySelector KeySelectors} defined for either both or
+	 * one input.
+	 *
+	 * <p>
+	 * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
+	 * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
+	 * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 */
+	public static class WithKey<T1, T2, KEY> {
+		DataStream<T1> input1;
+		DataStream<T2> input2;
+
+		KeySelector<T1, KEY> keySelector1;
+		KeySelector<T2, KEY> keySelector2;
+
+		protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
+			this.input1 = input1;
+			this.input2 = input2;
+
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the first input.
+		 */
+		public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector)  {
+			return new JoinedStreams.WithKey<>(input1, input2, keySelector, keySelector2);
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public JoinedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector)  {
+			return new JoinedStreams.WithKey<>(input1, input2, keySelector1, keySelector);
+		}
+
+		/**
+		 * Specifies the window on which the join operation works.
+		 */
+		public <W extends Window> JoinedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+			if (keySelector1 == null || keySelector2 == null) {
+				throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+
+			}
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
+		}
+	}
+
+	/**
+	 * A join operation that has {@link KeySelector KeySelectors} defined for both inputs as
+	 * well as a {@link WindowAssigner}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 * @param <W> Type of {@link Window} on which the join operation works.
+	 */
+	public static class WithWindow<T1, T2, KEY, W extends Window> {
+		private final DataStream<T1> input1;
+		private final DataStream<T2> input2;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+
+		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+		protected WithWindow(DataStream<T1> input1,
+				DataStream<T2> input2,
+				KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2,
+				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+			this.input1 = input1;
+			this.input2 = input2;
+
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+
+			this.windowAssigner = windowAssigner;
+			this.trigger = trigger;
+			this.evictor = evictor;
+		}
+
+		/**
+		 * Sets the {@code Trigger} that should be used to trigger window emission.
+		 */
+		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+		}
+
+		/**
+		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					JoinFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"CoGroup",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			return input1.coGroup(input2)
+					.where(keySelector1)
+					.equalTo(keySelector2)
+					.window(windowAssigner)
+					.trigger(trigger)
+					.evictor(evictor)
+					.apply(new FlatJoinCoGroupFunction<>(function), resultType);
+
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					JoinFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"CoGroup",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			return input1.coGroup(input2)
+					.where(keySelector1)
+					.equalTo(keySelector2)
+					.window(windowAssigner)
+					.trigger(trigger)
+					.evictor(evictor)
+					.apply(new JoinCoGroupFunction<>(function), resultType);
+
+		}
+	}
+
+	/**
+	 * Creates a new join operation from the two given inputs.
+	 */
+	public static <T1, T2> Unspecified<T1, T2> createJoin(DataStream<T1> input1, DataStream<T2> input2) {
+		return new Unspecified<>(input1, input2);
+	}
+
+	/**
+	 * CoGroup function that does a nested-loop join to get the join result.
+	 */
+	private static class JoinCoGroupFunction<T1, T2, T>
+			extends WrappingFunction<JoinFunction<T1, T2, T>>
+			implements CoGroupFunction<T1, T2, T> {
+		private static final long serialVersionUID = 1L;
+
+		public JoinCoGroupFunction(JoinFunction<T1, T2, T> wrappedFunction) {
+			super(wrappedFunction);
+		}
+
+		@Override
+		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+			for (T1 val1: first) {
+				for (T2 val2: second) {
+					out.collect(wrappedFunction.join(val1, val2));
+				}
+			}
+		}
+	}
+
+	/**
+	 * CoGroup function that does a nested-loop join to get the join result. (FlatJoin version)
+	 */
+	private static class FlatJoinCoGroupFunction<T1, T2, T>
+			extends WrappingFunction<FlatJoinFunction<T1, T2, T>>
+			implements CoGroupFunction<T1, T2, T> {
+		private static final long serialVersionUID = 1L;
+
+		public FlatJoinCoGroupFunction(FlatJoinFunction<T1, T2, T> wrappedFunction) {
+			super(wrappedFunction);
+		}
+
+		@Override
+		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+			for (T1 val1: first) {
+				for (T2 val2: second) {
+					wrappedFunction.join(val1, val2, out);
+				}
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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
deleted file mode 100644
index 4a5622d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
+++ /dev/null
@@ -1,274 +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.JoinFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-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.TypeExtractor;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-
-public class StreamJoinOperator<I1, I2> extends
-		TemporalOperator<I1, I2, StreamJoinOperator.JoinWindow<I1, I2>> {
-
-	public StreamJoinOperator(DataStream<I1> input1, DataStream<I2> input2) {
-		super(input1, input2);
-	}
-
-	@Override
-	protected JoinWindow<I1, I2> createNextWindowOperator() {
-		return new JoinWindow<I1, I2>(this);
-	}
-
-	public static class JoinWindow<I1, I2> implements TemporalWindow<JoinWindow<I1, I2>> {
-
-		private StreamJoinOperator<I1, I2> op;
-		private TypeInformation<I1> type1;
-
-		private JoinWindow(StreamJoinOperator<I1, I2> operator) {
-			this.op = operator;
-			this.type1 = op.input1.getType();
-		}
-
-		/**
-		 * Continues a temporal Join transformation. <br/>
-		 * Defines the {@link Tuple} fields of the first join {@link DataStream}
-		 * that should be used as join keys.<br/>
-		 * <b>Note: Fields can only be selected as join keys on Tuple
-		 * DataStreams.</b><br/>
-		 * 
-		 * @param fields
-		 *            The indexes of the other Tuple fields of the first join
-		 *            DataStreams that should be used as keys.
-		 * @return An incomplete Join transformation. Call
-		 *         {@link JoinPredicate#equalTo} to continue the Join.
-		 */
-		public JoinPredicate<I1, I2> where(int... fields) {
-			return new JoinPredicate<I1, I2>(op, KeySelectorUtil.getSelectorForKeys(
-					new Keys.ExpressionKeys<I1>(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig()));
-		}
-
-		/**
-		 * Continues a temporal join transformation. <br/>
-		 * Defines the fields of the first join {@link DataStream} that should
-		 * be used as grouping keys. Fields are the names of member fields of
-		 * the underlying type of the data stream.
-		 * 
-		 * @param fields
-		 *            The fields of the first join DataStream that should be
-		 *            used as keys.
-		 * @return An incomplete Join transformation. Call
-		 *         {@link JoinPredicate#equalTo} to continue the Join.
-		 */
-		public JoinPredicate<I1, I2> where(String... fields) {
-			return new JoinPredicate<I1, I2>(op, KeySelectorUtil.getSelectorForKeys(
-					new Keys.ExpressionKeys<I1>(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig()));
-		}
-
-		/**
-		 * Continues a temporal Join transformation and defines a
-		 * {@link KeySelector} function for the first join {@link DataStream}
-		 * .</br> The KeySelector function is called for each element of the
-		 * first DataStream and extracts a single key value on which the
-		 * DataStream is joined. </br>
-		 * 
-		 * @param keySelector
-		 *            The KeySelector function which extracts the key values
-		 *            from the DataStream on which it is joined.
-		 * @return An incomplete Join transformation. Call
-		 *         {@link JoinPredicate#equalTo} to continue the Join.
-		 */
-		public <K> JoinPredicate<I1, I2> where(KeySelector<I1, K> keySelector) {
-			return new JoinPredicate<I1, I2>(op, keySelector);
-		}
-
-		@Override
-		public JoinWindow<I1, I2> every(long length, TimeUnit timeUnit) {
-			return every(timeUnit.toMillis(length));
-		}
-
-		@Override
-		public JoinWindow<I1, I2> every(long length) {
-			op.slideInterval = length;
-			return this;
-		}
-
-		// ----------------------------------------------------------------------------------------
-
-	}
-
-	/**
-	 * Intermediate step of a temporal Join transformation. <br/>
-	 * To continue the Join transformation, select the join key of the second
-	 * input {@link DataStream} by calling {@link JoinPredicate#equalTo}
-	 * 
-	 */
-	public static class JoinPredicate<I1, I2> {
-
-		private StreamJoinOperator<I1, I2> op;
-		private KeySelector<I1, ?> keys1;
-		private KeySelector<I2, ?> keys2;
-		private TypeInformation<I2> type2;
-
-		private JoinPredicate(StreamJoinOperator<I1, I2> operator, KeySelector<I1, ?> keys1) {
-			this.op = operator;
-			this.keys1 = keys1;
-			this.type2 = op.input2.getType();
-		}
-
-		/**
-		 * Creates a temporal Join transformation and defines the {@link Tuple}
-		 * fields of the second join {@link DataStream} that should be used as
-		 * join keys.<br/>
-		 * </p> The resulting operator wraps each pair of joining elements in a
-		 * Tuple2<I1,I2>(first, second). To use a different wrapping function
-		 * use {@link JoinedStream#with(JoinFunction)}
-		 * 
-		 * @param fields
-		 *            The indexes of the Tuple fields of the second join
-		 *            DataStream that should be used as keys.
-		 * @return A streaming join operator. Call {@link JoinedStream#with} to
-		 *         apply a custom wrapping
-		 */
-		public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(int... fields) {
-			keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields, type2),
-					type2, op.input1.getExecutionEnvironment().getConfig());
-			return createJoinOperator();
-		}
-
-		/**
-		 * Creates a temporal Join transformation and defines the fields of the
-		 * second join {@link DataStream} that should be used as join keys. </p>
-		 * The resulting operator wraps each pair of joining elements in a
-		 * Tuple2<I1,I2>(first, second). To use a different wrapping function
-		 * use {@link JoinedStream#with(JoinFunction)}
-		 * 
-		 * @param fields
-		 *            The fields of the second join DataStream that should be
-		 *            used as keys.
-		 * @return A streaming join operator. Call {@link JoinedStream#with} to
-		 *         apply a custom wrapping
-		 */
-		public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(String... fields) {
-			this.keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields,
-					type2), type2, op.input1.getExecutionEnvironment().getConfig());
-			return createJoinOperator();
-		}
-
-		/**
-		 * Creates a temporal Join transformation and defines a
-		 * {@link KeySelector} function for the second join {@link DataStream}
-		 * .</br> The KeySelector function is called for each element of the
-		 * second DataStream and extracts a single key value on which the
-		 * DataStream is joined. </p> The resulting operator wraps each pair of
-		 * joining elements in a Tuple2<I1,I2>(first, second). To use a
-		 * different wrapping function use
-		 * {@link JoinedStream#with(JoinFunction)}
-		 * 
-		 * 
-		 * @param keySelector
-		 *            The KeySelector function which extracts the key values
-		 *            from the second DataStream on which it is joined.
-		 * @return A streaming join operator. Call {@link JoinedStream#with} to
-		 *         apply a custom wrapping
-		 */
-		public <K> JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(KeySelector<I2, K> keySelector) {
-			this.keys2 = keySelector;
-			return createJoinOperator();
-		}
-
-		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
-//					.keyBy(keys1)
-//					.connect(op.input2.keyBy(keys2))
-//					.addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
-//							op.slideInterval, op.timeStamp1, op.timeStamp2));
-			return null;
-		}
-
-		public static class JoinedStream<I1, I2, R> extends
-				SingleOutputStreamOperator<R, JoinedStream<I1, I2, R>> {
-			private final JoinPredicate<I1, I2> predicate;
-
-			private JoinedStream(JoinPredicate<I1, I2> predicate, DataStream<R> ds) {
-				super(ds.getExecutionEnvironment(), ds.getTransformation());
-				this.predicate = predicate;
-			}
-
-			/**
-			 * Completes a stream join. </p> The resulting operator wraps each pair
-			 * of joining elements using the user defined {@link JoinFunction}
-			 *
-			 * @return The joined data stream.
-			 */
-			@SuppressWarnings("unchecked")
-			public <OUT> SingleOutputStreamOperator<OUT, ?> with(JoinFunction<I1, I2, OUT> joinFunction) {
-
-				TypeInformation<OUT> outType = TypeExtractor.getJoinReturnTypes(joinFunction,
-						predicate.op.input1.getType(), predicate.op.input2.getType());
-
-//				JoinWindowFunction<I1, I2, OUT> joinWindowFunction = getJoinWindowFunction(joinFunction, predicate);
-//
-
-//				return new JoinedStream<I1, I2, OUT>(
-//						predicate, predicate.op.input1
-//						.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;
-			}
-		}
-	}
-
-	public static final class DefaultJoinFunction<I1, I2> implements
-			JoinFunction<I1, I2, Tuple2<I1, I2>> {
-
-		private static final long serialVersionUID = 1L;
-		private final Tuple2<I1, I2> outTuple = new Tuple2<I1, I2>();
-
-		@Override
-		public Tuple2<I1, I2> join(I1 first, I2 second) throws Exception {
-			outTuple.f0 = first;
-			outTuple.f1 = second;
-			return outTuple;
-		}
-	}
-
-//	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/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
deleted file mode 100644
index 9da00f2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
+++ /dev/null
@@ -1,124 +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.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-public abstract class TemporalOperator<I1, I2, OP extends TemporalWindow<OP>> {
-
-	public final DataStream<I1> input1;
-	public final DataStream<I2> input2;
-
-	public long windowSize;
-	public long slideInterval;
-
-	public TimestampWrapper<I1> timeStamp1;
-	public TimestampWrapper<I2> timeStamp2;
-
-	public TemporalOperator(DataStream<I1> input1, DataStream<I2> input2) {
-		if (input1 == null || input2 == null) {
-			throw new NullPointerException();
-		}
-		this.input1 = input1;
-		this.input2 = input2;
-	}
-
-	/**
-	 * Continues a temporal transformation.<br/>
-	 * Defines the window size on which the two DataStreams will be transformed.
-	 * To define sliding windows call {@link TemporalWindow#every} on the
-	 * resulting operator.
-	 * 
-	 * @param length
-	 *            The size of the window in milliseconds.
-	 * @param timeUnit
-	 *            The unit if time to be used
-	 * @return An incomplete temporal transformation.
-	 */
-	@SuppressWarnings("unchecked")
-	public OP onWindow(long length, TimeUnit timeUnit) {
-		return onWindow(timeUnit.toMillis(length),
-				(TimestampWrapper<I1>) SystemTimestamp.getWrapper(),
-				(TimestampWrapper<I2>) SystemTimestamp.getWrapper());
-	}
-
-	/**
-	 * Continues a temporal transformation.<br/>
-	 * Defines the window size on which the two DataStreams will be
-	 * transformed.To define sliding windows call {@link TemporalWindow#every}
-	 * on the resulting operator.
-	 * 
-	 * @param length
-	 *            The size of the window in milliseconds.
-	 * @param timeStamp1
-	 *            The timestamp used to extract time from the elements of the
-	 *            first data stream.
-	 * @param timeStamp2
-	 *            The timestamp used to extract time from the elements of the
-	 *            second data stream.
-	 * @return An incomplete temporal transformation.
-	 */
-	public OP onWindow(long length, Timestamp<I1> timeStamp1, Timestamp<I2> timeStamp2) {
-		return onWindow(length, timeStamp1, timeStamp2, 0);
-	}
-
-	/**
-	 * Continues a temporal transformation.<br/>
-	 * Defines the window size on which the two DataStreams will be
-	 * transformed.To define sliding windows call {@link TemporalWindow#every}
-	 * on the resulting operator.
-	 * 
-	 * @param length
-	 *            The size of the window in milliseconds.
-	 * @param timeStamp1
-	 *            The timestamp used to extract time from the elements of the
-	 *            first data stream.
-	 * @param timeStamp2
-	 *            The timestamp used to extract time from the elements of the
-	 *            second data stream.
-	 * @param startTime
-	 *            The start time to measure the first window
-	 * @return An incomplete temporal transformation.
-	 */
-	public OP onWindow(long length, Timestamp<I1> timeStamp1, Timestamp<I2> timeStamp2,
-			long startTime) {
-		return onWindow(length, new TimestampWrapper<I1>(timeStamp1, startTime),
-				new TimestampWrapper<I2>(timeStamp2, startTime));
-	}
-
-	private OP onWindow(long length, TimestampWrapper<I1> timeStamp1,
-			TimestampWrapper<I2> timeStamp2) {
-
-		this.windowSize = length;
-		this.slideInterval = length;
-
-		this.timeStamp1 = timeStamp1;
-		this.timeStamp2 = timeStamp2;
-
-		return createNextWindowOperator();
-	}
-
-	protected abstract OP createNextWindowOperator();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java
deleted file mode 100644
index 49c75c4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.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.datastream.temporal;
-
-import java.util.concurrent.TimeUnit;
-
-public interface TemporalWindow<T> {
-
-	/**
-	 * Defines the slide interval for this temporal operator
-	 * 
-	 * @param length
-	 *            Length of the window
-	 * @param timeUnit
-	 *            Unit of time
-	 * @return The temporal operator with slide interval specified
-	 */
-	public T every(long length, TimeUnit timeUnit);
-
-	/**
-	 * Defines the slide interval for this temporal operator
-	 * 
-	 * @param length
-	 *            Length of the window
-	 * @return The temporal operator with slide interval specified
-	 */
-	public T every(long length);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 042fe18..edd8a34 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
@@ -18,37 +18,17 @@
 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.api.java.operators.translation.WrappingFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
-public class ReduceWindowFunction<K, W extends Window, T> extends RichWindowFunction<T, T, K, W> {
+public class ReduceWindowFunction<K, W extends Window, T>
+		extends WrappingFunction<ReduceFunction<T>>
+		implements WindowFunction<T, T, K, W> {
 	private static final long serialVersionUID = 1L;
 
-	private final ReduceFunction<T> reduceFunction;
-
 	public ReduceWindowFunction(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);
+		super(reduceFunction);
 	}
 
 	@Override
@@ -59,7 +39,7 @@ public class ReduceWindowFunction<K, W extends Window, T> extends RichWindowFunc
 			if (result == null) {
 				result = v;
 			} else {
-				result = reduceFunction.reduce(result, v);
+				result = wrappedFunction.reduce(result, v);
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
new file mode 100644
index 0000000..c06a608
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
@@ -0,0 +1,372 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.flink.streaming.api;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+	private static List<String> testResults;
+
+	@Test
+	public void testCoGroup() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+		env.getConfig().enableTimestamps();
+
+		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+				ctx.collect(Tuple2.of("a", 2));
+
+				ctx.collect(Tuple2.of("b", 3));
+				ctx.collect(Tuple2.of("b", 4));
+				ctx.collect(Tuple2.of("b", 5));
+
+				ctx.collect(Tuple2.of("a", 6));
+				ctx.collect(Tuple2.of("a", 7));
+				ctx.collect(Tuple2.of("a", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+		DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+
+				ctx.collect(Tuple2.of("b", 3));
+
+				ctx.collect(Tuple2.of("c", 6));
+				ctx.collect(Tuple2.of("c", 7));
+				ctx.collect(Tuple2.of("c", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+
+		source1.coGroup(source2)
+				.where(new Tuple2KeyExtractor())
+				.equalTo(new Tuple2KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+					@Override
+					public void coGroup(Iterable<Tuple2<String, Integer>> first,
+							Iterable<Tuple2<String, Integer>> second,
+							Collector<String> out) throws Exception {
+						StringBuilder result = new StringBuilder();
+						result.append("F:");
+						for (Tuple2<String, Integer> t: first) {
+							result.append(t.toString());
+						}
+						result.append(" S:");
+						for (Tuple2<String, Integer> t: second) {
+							result.append(t.toString());
+						}
+						out.collect(result.toString());
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("CoGroup Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+				"F:(b,3)(b,4)(b,5) S:(b,3)",
+				"F:(a,6)(a,7)(a,8) S:",
+				"F: S:(c,6)(c,7)(c,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	@Test
+	public void testJoin() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+		env.getConfig().enableTimestamps();
+
+		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "x", 0));
+				ctx.collect(Tuple3.of("a", "y", 1));
+				ctx.collect(Tuple3.of("a", "z", 2));
+
+				ctx.collect(Tuple3.of("b", "u", 3));
+				ctx.collect(Tuple3.of("b", "w", 5));
+
+				ctx.collect(Tuple3.of("a", "i", 6));
+				ctx.collect(Tuple3.of("a", "j", 7));
+				ctx.collect(Tuple3.of("a", "k", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+		DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "u", 0));
+				ctx.collect(Tuple3.of("a", "w", 1));
+
+				ctx.collect(Tuple3.of("b", "i", 3));
+				ctx.collect(Tuple3.of("b", "k", 5));
+
+				ctx.collect(Tuple3.of("a", "x", 6));
+				ctx.collect(Tuple3.of("a", "z", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+
+		source1.join(source2)
+				.where(new Tuple3KeyExtractor())
+				.equalTo(new Tuple3KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+					@Override
+					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+						return first + ":" + second;
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("Join Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(a,x,0):(a,u,0)",
+				"(a,x,0):(a,w,1)",
+				"(a,y,1):(a,u,0)",
+				"(a,y,1):(a,w,1)",
+				"(a,z,2):(a,u,0)",
+				"(a,z,2):(a,w,1)",
+				"(b,u,3):(b,i,3)",
+				"(b,u,3):(b,k,5)",
+				"(b,w,5):(b,i,3)",
+				"(b,w,5):(b,k,5)",
+				"(a,i,6):(a,x,6)",
+				"(a,i,6):(a,z,8)",
+				"(a,j,7):(a,x,6)",
+				"(a,j,7):(a,z,8)",
+				"(a,k,8):(a,x,6)",
+				"(a,k,8):(a,z,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	@Test
+	public void testSelfJoin() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+		env.getConfig().enableTimestamps();
+
+		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "x", 0));
+				ctx.collect(Tuple3.of("a", "y", 1));
+				ctx.collect(Tuple3.of("a", "z", 2));
+
+				ctx.collect(Tuple3.of("b", "u", 3));
+				ctx.collect(Tuple3.of("b", "w", 5));
+
+				ctx.collect(Tuple3.of("a", "i", 6));
+				ctx.collect(Tuple3.of("a", "j", 7));
+				ctx.collect(Tuple3.of("a", "k", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+		source1.join(source1)
+				.where(new Tuple3KeyExtractor())
+				.equalTo(new Tuple3KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+					@Override
+					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+						return first + ":" + second;
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("Self-Join Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(a,x,0):(a,x,0)",
+				"(a,x,0):(a,y,1)",
+				"(a,x,0):(a,z,2)",
+				"(a,y,1):(a,x,0)",
+				"(a,y,1):(a,y,1)",
+				"(a,y,1):(a,z,2)",
+				"(a,z,2):(a,x,0)",
+				"(a,z,2):(a,y,1)",
+				"(a,z,2):(a,z,2)",
+				"(b,u,3):(b,u,3)",
+				"(b,u,3):(b,w,5)",
+				"(b,w,5):(b,u,3)",
+				"(b,w,5):(b,w,5)",
+				"(a,i,6):(a,i,6)",
+				"(a,i,6):(a,j,7)",
+				"(a,i,6):(a,k,8)",
+				"(a,j,7):(a,i,6)",
+				"(a,j,7):(a,j,7)",
+				"(a,j,7):(a,k,8)",
+				"(a,k,8):(a,i,6)",
+				"(a,k,8):(a,j,7)",
+				"(a,k,8):(a,k,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1;
+		}
+
+		@Override
+		public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
+			return element.f2;
+		}
+
+		@Override
+		public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
+			return element.f2 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class Tuple2KeyExtractor 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;
+		}
+	}
+
+	private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple3<String, String, Integer> value) throws Exception {
+			return value.f0;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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
deleted file mode 100644
index 7d2a131..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
+++ /dev/null
@@ -1,143 +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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashSet;
-
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
-
-	private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> joinExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
-	private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> crossExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
-
-	private static class MyTimestamp<T> implements Timestamp<T> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long getTimestamp(T value) {
-			return 101L;
-		}
-	}
-
-	/**
-	 * TODO: enable once new join operator is ready
-	 * @throws Exception
-	 */
-	@Ignore
-	@Test
-	public void test() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-		env.setBufferTimeout(1);
-
-		TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>> joinResultSink =
-				new TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>>();
-		TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>> crossResultSink =
-				new TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>>();
-
-		ArrayList<Tuple2<Integer, String>> in1 = new ArrayList<Tuple2<Integer, String>>();
-		ArrayList<Tuple1<Integer>> in2 = new ArrayList<Tuple1<Integer>>();
-
-		in1.add(new Tuple2<Integer, String>(10, "a"));
-		in1.add(new Tuple2<Integer, String>(20, "b"));
-		in1.add(new Tuple2<Integer, String>(20, "x"));
-		in1.add(new Tuple2<Integer, String>(0, "y"));
-
-		in2.add(new Tuple1<Integer>(0));
-		in2.add(new Tuple1<Integer>(5));
-		in2.add(new Tuple1<Integer>(20));
-
-		joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "b"), 20));
-		joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "x"), 20));
-		joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(0, "y"), 0));
-
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(10, "a"), 0));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(10, "a"), 5));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(10, "a"), 20));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "b"), 0));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "b"), 5));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "b"), 20));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "x"), 0));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "x"), 5));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(20, "x"), 20));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(0, "y"), 0));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(0, "y"), 5));
-		crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
-				new Tuple2<Integer, String>(0, "y"), 20));
-
-		DataStream<Tuple2<Integer, String>> inStream1 = env.fromCollection(in1);
-		DataStream<Tuple1<Integer>> inStream2 = env.fromCollection(in2);
-
-		inStream1
-				.join(inStream2)
-				.onWindow(1000, new MyTimestamp<Tuple2<Integer, String>>(),
-						new MyTimestamp<Tuple1<Integer>>(), 100).where(0).equalTo(0)
-				.map(new ResultMap())
-				.addSink(joinResultSink);
-
-		env.execute();
-
-		assertEquals(new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(joinExpectedResults),
-				new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(joinResultSink.getResult()));
-		assertEquals(new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(crossExpectedResults),
-				new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(crossResultSink.getResult()));
-	}
-
-	private static class ResultMap implements
-			MapFunction<Tuple2<Tuple2<Integer, String>, Tuple1<Integer>>,
-					Tuple2<Tuple2<Integer, String>, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Tuple2<Integer, String>, Integer> map(Tuple2<Tuple2<Integer, String>, Tuple1<Integer>> value) throws Exception {
-			return new Tuple2<Tuple2<Integer, String>, Integer>(value.f0, value.f1.f0);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 fc9de1d..c116c01 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
@@ -47,67 +47,6 @@ 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() {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		TestListResultSink<String> resultSink = new TestListResultSink<String>();
-
-		Timestamp<Integer> timeStamp = new IntegerTimestamp();
-
-		KeySelector keySelector = new KeySelector<Integer, Integer>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer getKey(Integer value) throws Exception {
-				return value;
-			}
-		};
-
-		DataStream<Integer> src = env.fromElements(1, 3, 5);
-
-		@SuppressWarnings("unused")
-		DataStreamSink<Tuple2<Integer, Integer>> dataStream =
-				src.join(src).onWindow(50L, timeStamp, timeStamp).where(keySelector).equalTo(keySelector)
-						.map(new MapFunction<Tuple2<Integer, Integer>, String>() {
-
-							private static final long serialVersionUID = 1L;
-
-							@Override
-							public String map(Tuple2<Integer, Integer> value) throws Exception {
-								return value.toString();
-							}
-						})
-						.addSink(resultSink);
-
-
-		try {
-			env.execute();
-
-			expected = new ArrayList<String>();
-
-			expected.addAll(Arrays.asList("(1,1)", "(3,3)", "(5,5)"));
-
-			List<String> result = resultSink.getResult();
-
-			Collections.sort(expected);
-			Collections.sort(result);
-
-			assertEquals(expected, result);
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail();
-		}
-	}
-
-	/**
 	 * We connect two different data streams in a chain to a CoMap.
 	 */
 	@Test


[8/8] flink git commit: [FLINK-2550] Add Window Aggregations to new Windowing API

Posted by al...@apache.org.
[FLINK-2550] Add Window Aggregations to new Windowing API


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

Branch: refs/heads/master
Commit: 28a38bb7dedbc10ceab9d4ae1dbcc15789e33211
Parents: 0bac272
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 17:37:39 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       | 205 +++++++++++++++++++
 .../api/datastream/WindowedStream.java          | 205 +++++++++++++++++++
 .../aggregation/ComparableAggregator.java       |  11 +-
 .../functions/aggregation/SumAggregator.java    |   2 +-
 .../windowing/util/SessionWindowingData.java    |   2 +-
 .../streaming/api/scala/AllWindowedStream.scala |  96 +++++++++
 .../streaming/api/scala/WindowedStream.scala    |  96 +++++++++
 .../StreamingScalaAPICompletenessTest.scala     |   3 +
 8 files changed, 612 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/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 0cc1854..89c4857 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
@@ -25,6 +25,9 @@ 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.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.windowing.ReduceAllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -222,6 +225,204 @@ public class AllWindowedStream<T, W extends Window> {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Aggregations on the  windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies an aggregation that sums every window of the data stream at the
+	 * given position.
+	 *
+	 * @param positionToSum The position in the tuple/array to sum
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that sums every window of the pojo data stream at
+	 * the given field for every window.
+	 *
+	 * <p>
+	 * A field expression is either
+	 * the name of a public field or a getter method with parentheses of the
+	 * stream's underlying type. A dot can be used to drill down into objects,
+	 * as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field to sum
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> sum(String field) {
+		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of every window
+	 * of the data stream at the given position.
+	 *
+	 * @param positionToMin The position to minimize
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of the pojo data
+	 * stream at the given field expression for every window.
+	 *
+	 * <p>
+	 * 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 DataStream<T> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMinBy The position to minimize
+	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream DataStreams} 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 DataStream<T> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum value of every window of
+	 * the data stream at the given position.
+	 *
+	 * @param positionToMax The position to maximize
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum value of the pojo data
+	 * stream at the given field expression for every window. A field expression
+	 * is either the name of a public field or a getter method with parentheses
+	 * of the {@link DataStream DataStreams} 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 DataStream<T> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMaxBy The position to maximize by
+	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum element of the pojo
+	 * data stream by the given field expression for every window. 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 DataStream<T> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+		return reduce(aggregator);
+	}
+
+	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 
@@ -238,4 +439,8 @@ public class AllWindowedStream<T, W extends Window> {
 	public StreamExecutionEnvironment getExecutionEnvironment() {
 		return input.getExecutionEnvironment();
 	}
+
+	public TypeInformation<T> getInputType() {
+		return input.getType();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/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 0ea9cad..1273b42 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
@@ -26,6 +26,9 @@ 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.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.windowing.WindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -241,6 +244,204 @@ public class WindowedStream<T, K, W extends Window> {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Aggregations on the keyed windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies an aggregation that sums every window of the data stream at the
+	 * given position.
+	 *
+	 * @param positionToSum The position in the tuple/array to sum
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that sums every window of the pojo data stream at
+	 * the given field for every window.
+	 *
+	 * <p>
+	 * A field expression is either
+	 * the name of a public field or a getter method with parentheses of the
+	 * stream's underlying type. A dot can be used to drill down into objects,
+	 * as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field to sum
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> sum(String field) {
+		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of every window
+	 * of the data stream at the given position.
+	 *
+	 * @param positionToMin The position to minimize
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of the pojo data
+	 * stream at the given field expression for every window.
+	 *
+	 * <p>
+	 * 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 DataStream<T> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMinBy The position to minimize
+	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream DataStreams} 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 DataStream<T> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum value of every window of
+	 * the data stream at the given position.
+	 *
+	 * @param positionToMax The position to maximize
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum value of the pojo data
+	 * stream at the given field expression for every window. A field expression
+	 * is either the name of a public field or a getter method with parentheses
+	 * of the {@link DataStream DataStreams} 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 DataStream<T> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMaxBy The position to maximize by
+	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum element of the pojo
+	 * data stream by the given field expression for every window. 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 DataStream<T> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+		return reduce(aggregator);
+	}
+
+	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 
@@ -307,4 +508,8 @@ public class WindowedStream<T, K, W extends Window> {
 	public StreamExecutionEnvironment getExecutionEnvironment() {
 		return input.getExecutionEnvironment();
 	}
+
+	public TypeInformation<T> getInputType() {
+		return input.getType();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
index 766a59e..e5501a0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
@@ -33,8 +33,7 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
 	private ComparableAggregator(int pos, AggregationType aggregationType, boolean first) {
 		super(pos);
 		this.comparator = Comparator.getForAggregation(aggregationType);
-		this.byAggregate = (aggregationType == AggregationType.MAXBY)
-				|| (aggregationType == AggregationType.MINBY);
+		this.byAggregate = (aggregationType == AggregationType.MAXBY) || (aggregationType == AggregationType.MINBY);
 		this.first = first;
 	}
 
@@ -61,7 +60,7 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
 	@SuppressWarnings("unchecked")
 	@Override
 	public T reduce(T value1, T value2) throws Exception {
-		Comparable<Object> o1 = (Comparable<Object>)fieldAccessor.get(value1);
+		Comparable<Object> o1 = (Comparable<Object>) fieldAccessor.get(value1);
 		Object o2 = fieldAccessor.get(value2);
 
 		int c = comparator.isExtremal(o1, o2);
@@ -79,10 +78,10 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
 			return value2;
 
 		} else {
-			if (c == 1) {
-				value2 = fieldAccessor.set(value2, o1);
+			if (c == 0) {
+				value1 = fieldAccessor.set(value1, o2);
 			}
-			return value2;
+			return value1;
 		}
 
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
index c23695e..b045233 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
@@ -43,6 +43,6 @@ public class SumAggregator<T> extends AggregationFunction<T> {
 	@SuppressWarnings("unchecked")
 	@Override
 	public T reduce(T value1, T value2) throws Exception {
-		return fieldAccessor.set(value2, adder.add(fieldAccessor.get(value1), fieldAccessor.get(value2)));
+		return fieldAccessor.set(value1, adder.add(fieldAccessor.get(value1), fieldAccessor.get(value2)));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
index bb4a123..c1a99a8 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.examples.windowing.util;
 
 public class SessionWindowingData {
 
-	public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,5,3)\n" +
+	public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" +
 			"(a,10,1)";
 
 	private SessionWindowingData() {

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/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
index 9054d95..d2d0a1d 100644
--- 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
@@ -21,6 +21,8 @@ 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.aggregation.AggregationFunction.AggregationType
+import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
 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
@@ -134,6 +136,96 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
     javaStream.apply(clean(function), implicitly[TypeInformation[R]])
   }
 
+  // ------------------------------------------------------------------------
+  //  Aggregations on the keyed windows
+  // ------------------------------------------------------------------------
+
+  /**
+   * Applies an aggregation that that gives the maximum of the elements in the window at
+   * the given position.
+   */
+  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
+
+  /**
+   * Applies an aggregation that that gives the maximum of the elements in the window at
+   * the given field.
+   */
+  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
+
+  /**
+   * Applies an aggregation that that gives the minimum of the elements in the window at
+   * the given position.
+   */
+  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
+
+  /**
+   * Applies an aggregation that that gives the minimum of the elements in the window at
+   * the given field.
+   */
+  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
+
+  /**
+   * Applies an aggregation that sums the elements in the window at the given position.
+   */
+  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
+
+  /**
+   * Applies an aggregation that sums the elements in the window at the given field.
+   */
+  def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
+
+  /**
+   * Applies an aggregation that that gives the maximum element of the window by
+   * the given position. When equality, returns the first.
+   */
+  def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
+    position)
+
+  /**
+   * Applies an aggregation that that gives the maximum element of the window by
+   * the given field. When equality, returns the first.
+   */
+  def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
+    field)
+
+  /**
+   * Applies an aggregation that that gives the minimum element of the window by
+   * the given position. When equality, returns the first.
+   */
+  def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
+    position)
+
+  /**
+   * Applies an aggregation that that gives the minimum element of the window by
+   * the given field. When equality, returns the first.
+   */
+  def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
+    field)
+
+  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
+    val position = fieldNames2Indices(getInputType(), Array(field))(0)
+    aggregate(aggregationType, position)
+  }
+
+  def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
+
+    val jStream = javaStream.asInstanceOf[JavaAllWStream[Product, W]]
+
+    val reducer = aggregationType match {
+      case AggregationType.SUM =>
+        new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
+
+      case _ =>
+        new ComparableAggregator(
+          position,
+          jStream.getInputType,
+          aggregationType,
+          true,
+          jStream.getExecutionEnvironment.getConfig)
+    }
+
+    new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
+  }
 
   // ------------------------------------------------------------------------
   //  Utilities
@@ -147,4 +239,8 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
     new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
   }
 
+  /**
+   * Gets the output type.
+   */
+  private def getInputType(): TypeInformation[T] = javaStream.getInputType
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/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
index 2d6806d..3963765 100644
--- 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
@@ -21,6 +21,8 @@ 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.aggregation.AggregationFunction.AggregationType
+import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
 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
@@ -137,6 +139,96 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
     javaStream.apply(clean(function), implicitly[TypeInformation[R]])
   }
 
+  // ------------------------------------------------------------------------
+  //  Aggregations on the keyed windows
+  // ------------------------------------------------------------------------
+
+  /**
+   * Applies an aggregation that that gives the maximum of the elements in the window at
+   * the given position.
+   */
+  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
+
+  /**
+   * Applies an aggregation that that gives the maximum of the elements in the window at
+   * the given field.
+   */
+  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
+
+  /**
+   * Applies an aggregation that that gives the minimum of the elements in the window at
+   * the given position.
+   */
+  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
+
+  /**
+   * Applies an aggregation that that gives the minimum of the elements in the window at
+   * the given field.
+   */
+  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
+
+  /**
+   * Applies an aggregation that sums the elements in the window at the given position.
+   */
+  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
+
+  /**
+   * Applies an aggregation that sums the elements in the window at the given field.
+   */
+  def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
+
+  /**
+   * Applies an aggregation that that gives the maximum element of the window by
+   * the given position. When equality, returns the first.
+   */
+  def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
+    position)
+
+  /**
+   * Applies an aggregation that that gives the maximum element of the window by
+   * the given field. When equality, returns the first.
+   */
+  def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
+    field)
+
+  /**
+   * Applies an aggregation that that gives the minimum element of the window by
+   * the given position. When equality, returns the first.
+   */
+  def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
+    position)
+
+  /**
+   * Applies an aggregation that that gives the minimum element of the window by
+   * the given field. When equality, returns the first.
+   */
+  def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
+    field)
+
+  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
+    val position = fieldNames2Indices(getInputType(), Array(field))(0)
+    aggregate(aggregationType, position)
+  }
+
+  def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
+
+    val jStream = javaStream.asInstanceOf[JavaWStream[Product, K, W]]
+
+    val reducer = aggregationType match {
+      case AggregationType.SUM =>
+        new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
+
+      case _ =>
+        new ComparableAggregator(
+          position,
+          jStream.getInputType,
+          aggregationType,
+          true,
+          jStream.getExecutionEnvironment.getConfig)
+    }
+
+    new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
+  }
 
   // ------------------------------------------------------------------------
   //  Utilities
@@ -150,4 +242,8 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
     new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
   }
 
+  /**
+   * Gets the output type.
+   */
+  private def getInputType(): TypeInformation[T] = javaStream.getInputType
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/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 c6bd87a..53aa1e2 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
@@ -51,11 +51,14 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "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",
 
       "org.apache.flink.streaming.api.datastream.WindowedStream.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.WindowedStream.getInputType",
       "org.apache.flink.streaming.api.datastream.AllWindowedStream.getExecutionEnvironment",
+      "org.apache.flink.streaming.api.datastream.AllWindowedStream.getInputType",
 
       "org.apache.flink.streaming.api.datastream.KeyedStream.transform",
       "org.apache.flink.streaming.api.datastream.KeyedStream.getKeySelector",


[2/8] flink git commit: [FLINK-2550] Rework interplay of Window Assigners and TimeCharacteristic

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 fa9c0a9..65f978c 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
@@ -23,7 +23,7 @@ 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.assigners.{TumblingTimeWindows, SlidingTimeWindows}
 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}
@@ -52,7 +52,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(
+      .window(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .reduce(reducer)
@@ -66,7 +66,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(
+      .window(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
@@ -95,7 +95,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(
+      .window(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
@@ -109,14 +109,14 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
     val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
     assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
     assertTrue(
       winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
 
 
     val window2 = source
       .keyBy(0)
-      .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
       def apply(
@@ -134,7 +134,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
     val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
     assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
     assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
   }
 
@@ -148,7 +148,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window1 = source
       .keyBy(0)
-      .window(SlidingProcessingTimeWindows.of(
+      .window(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
@@ -163,13 +163,13 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val winOperator1 = operator1.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
     assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
     assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
     assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
 
 
     val window2 = source
       .keyBy(0)
-      .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .evictor(CountEvictor.of(1000))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
@@ -189,7 +189,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val winOperator2 = operator2.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
     assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
     assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
     assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
   }
 }


[6/8] flink git commit: [FLINK-2550] Rename reduceWindow to reduce on *WindowedStream, add Lambda Reduce

Posted by al...@apache.org.
[FLINK-2550] Rename reduceWindow to reduce on *WindowedStream, add Lambda Reduce

Lambda Reduce is the reduce method that takes a Scala Lambda function.


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

Branch: refs/heads/master
Commit: 0bac272c8309f2e7567ba762076bd75eeb8ea83a
Parents: 8634dbb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 17:36:33 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       |  2 +-
 .../api/datastream/WindowedStream.java          |  2 +-
 .../windowing/AllWindowTranslationTest.java     |  6 ++--
 .../windowing/TimeWindowTranslationTest.java    |  4 +--
 .../windowing/WindowTranslationTest.java        |  6 ++--
 .../GroupedProcessingTimeWindowExample.java     |  2 +-
 .../streaming/api/scala/AllWindowedStream.scala | 31 ++++++++++++++++++--
 .../streaming/api/scala/WindowedStream.scala    | 31 ++++++++++++++++++--
 .../api/scala/AllWindowTranslationTest.scala    |  6 ++--
 .../api/scala/WindowTranslationTest.scala       |  6 ++--
 10 files changed, 75 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 78ba8ad..0cc1854 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
@@ -120,7 +120,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @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) {
+	public DataStream<T> reduce(ReduceFunction<T> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 349651e..0ea9cad 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
@@ -131,7 +131,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @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) {
+	public DataStream<T> reduce(ReduceFunction<T> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 767b40c..09a7149 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
@@ -66,7 +66,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduceWindow(reducer);
+				.reduce(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();
@@ -103,7 +103,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
-				.reduceWindow(reducer);
+				.reduce(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();
@@ -149,7 +149,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
-				.reduceWindow(reducer);
+				.reduce(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();

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 76d7bfe..76c6f20 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
@@ -59,7 +59,7 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
 				.timeWindow(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS))
-				.reduceWindow(reducer);
+				.reduce(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();
@@ -103,7 +103,7 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS),
 						Time.of(100, TimeUnit.MILLISECONDS))
-				.reduceWindow(reducer);
+				.reduce(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();

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 9dc6687..5124add 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
@@ -66,7 +66,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 				.keyBy(0)
 				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS),
 						Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduceWindow(reducer);
+				.reduce(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();
@@ -105,7 +105,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 				.keyBy(0)
 				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
-				.reduceWindow(reducer);
+				.reduce(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();
@@ -153,7 +153,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 				.keyBy(0)
 				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
-				.reduceWindow(reducer);
+				.reduce(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();

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 5d32b8e..982b73d 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
@@ -79,7 +79,7 @@ public class GroupedProcessingTimeWindowExample {
 		stream
 			.keyBy(0)
 			.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
-			.reduceWindow(new SummingReducer())
+			.reduce(new SummingReducer())
 
 			// alternative: use a apply function which does not pre-aggregate
 //			.keyBy(new FirstFieldKeyExtractor<Tuple2<Long, Long>, Long>())

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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
index 4f36722..9054d95 100644
--- 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
@@ -73,6 +73,26 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
   // ------------------------------------------------------------------------
   //  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 reduce(function: ReduceFunction[T]): DataStream[T] = {
+    javaStream.reduce(clean(function))
+  }
+
   /**
    * 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
@@ -88,8 +108,15 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
    * @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))
+  def reduce(function: (T, T) => T): DataStream[T] = {
+    if (function == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+    val cleanFun = clean(function)
+    val reducer = new ReduceFunction[T] {
+      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
+    }
+    reduce(reducer)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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
index a688846..2d6806d 100644
--- 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
@@ -76,6 +76,26 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
   // ------------------------------------------------------------------------
   //  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 reduce(function: ReduceFunction[T]): DataStream[T] = {
+    javaStream.reduce(clean(function))
+  }
+
   /**
    * 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
@@ -91,8 +111,15 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
    * @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))
+  def reduce(function: (T, T) => T): DataStream[T] = {
+    if (function == null) {
+      throw new NullPointerException("Reduce function must not be null.")
+    }
+    val cleanFun = clean(function)
+    val reducer = new ReduceFunction[T] {
+      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
+    }
+    reduce(reducer)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 247256f..dece9f6 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
@@ -58,7 +58,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .windowAll(SlidingProcessingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
       .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -100,7 +100,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
       .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -150,7 +150,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
       .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]

http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/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 f1b05c6..fa9c0a9 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
@@ -55,7 +55,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .window(SlidingProcessingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
         .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -99,7 +99,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
       .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -152,7 +152,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
-      .reduceWindow(reducer)
+      .reduce(reducer)
 
     val transform1 = window1.getJavaStream.getTransformation
       .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]


[4/8] flink git commit: [FLINK-2819] Add Windowed Join/CoGroup Operator Based on Tagged Union

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
index 5230e9b..8abf9d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
@@ -19,31 +19,36 @@ package org.apache.flink.streaming.examples.join;
 
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
 
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Example illustrating join over sliding windows of streams in Flink.
- * <p/>
+ *
  * <p>
- * his example will join two streams with a sliding window. One which emits
- * grades and one which emits salaries of people.
- * </p>
- * <p/>
- * <p/>
+ * This example will join two streams with a sliding window. One which emits grades and one which
+ * emits salaries of people. The input format for both sources has an additional timestamp
+ * as field 0. This is used to to event-time windowing. Time timestamps must be
+ * monotonically increasing.
+ *
  * This example shows how to:
  * <ul>
- * <li>do windowed joins,
- * <li>use tuple data types,
- * <li>write a simple streaming program.
+ *   <li>do windowed joins,
+ *   <li>use tuple data types,
+ *   <li>write a simple streaming program.
+ * </ul>
  */
 public class WindowJoin {
 
@@ -51,9 +56,6 @@ public class WindowJoin {
 	// PROGRAM
 	// *************************************************************************
 
-	private static DataStream<Tuple2<String, Integer>> grades;
-	private static DataStream<Tuple2<String, Integer>> salaries;
-
 	public static void main(String[] args) throws Exception {
 
 		if (!parseParameters(args)) {
@@ -62,18 +64,25 @@ public class WindowJoin {
 
 		// obtain execution environment
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableTimestamps();
 
 		// connect to the data sources for grades and salaries
-		setInputStreams(env);
+		Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);
+		DataStream<Tuple3<Long, String, Integer>> grades = input.f0;
+		DataStream<Tuple3<Long, String, Integer>> salaries = input.f1;
+
+		// extract the timestamps
+		grades = grades.extractTimestamp(new MyTimestampExtractor());
+		salaries = salaries.extractTimestamp(new MyTimestampExtractor());
 
 		// apply a temporal join over the two stream based on the names over one
 		// second windows
 		DataStream<Tuple3<String, Integer, Integer>> joinedStream = grades
 				.join(salaries)
-				.onWindow(1, new MyTimestamp(0), new MyTimestamp(0))
-				.where(0)
-				.equalTo(0)
-				.with(new MyJoinFunction());
+				.where(new NameKeySelector())
+				.equalTo(new NameKeySelector())
+				.window(TumblingTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
+				.apply(new MyJoinFunction());
 
 		// emit result
 		if (fileOutput) {
@@ -98,24 +107,25 @@ public class WindowJoin {
 	/**
 	 * Continuously emit tuples with random names and integers (grades).
 	 */
-	public static class GradeSource implements SourceFunction<Tuple2<String, Integer>> {
+	public static class GradeSource implements SourceFunction<Tuple3<Long, String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		private Random rand;
-		private Tuple2<String, Integer> outTuple;
+		private Tuple3<Long, String, Integer> outTuple;
 		private volatile boolean isRunning = true;
 		private int counter;
 
 		public GradeSource() {
 			rand = new Random();
-			outTuple = new Tuple2<String, Integer>();
+			outTuple = new Tuple3<>();
 		}
 
 		@Override
-		public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
 			while (isRunning && counter < 100) {
-				outTuple.f0 = names[rand.nextInt(names.length)];
-				outTuple.f1 = rand.nextInt(GRADE_COUNT) + 1;
+				outTuple.f0 = System.currentTimeMillis();
+				outTuple.f1 = names[rand.nextInt(names.length)];
+				outTuple.f2 = rand.nextInt(GRADE_COUNT) + 1;
 				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
 				counter++;
 				ctx.collect(outTuple);
@@ -131,27 +141,28 @@ public class WindowJoin {
 	/**
 	 * Continuously emit tuples with random names and integers (salaries).
 	 */
-	public static class SalarySource extends RichSourceFunction<Tuple2<String, Integer>> {
+	public static class SalarySource extends RichSourceFunction<Tuple3<Long, String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		private transient Random rand;
-		private transient Tuple2<String, Integer> outTuple;
+		private transient Tuple3<Long, String, Integer> outTuple;
 		private volatile boolean isRunning;
 		private int counter;
 
 		public void open(Configuration parameters) throws Exception {
 			super.open(parameters);
 			rand = new Random();
-			outTuple = new Tuple2<String, Integer>();
+			outTuple = new Tuple3<Long, String, Integer>();
 			isRunning = true;
 		}
 
 
 		@Override
-		public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
 			while (isRunning && counter < 100) {
-				outTuple.f0 = names[rand.nextInt(names.length)];
-				outTuple.f1 = rand.nextInt(SALARY_MAX) + 1;
+				outTuple.f0 = System.currentTimeMillis();
+				outTuple.f1 = names[rand.nextInt(names.length)];
+				outTuple.f2 = rand.nextInt(SALARY_MAX) + 1;
 				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
 				counter++;
 				ctx.collect(outTuple);
@@ -164,7 +175,7 @@ public class WindowJoin {
 		}
 	}
 
-	public static class MySourceMap extends RichMapFunction<String, Tuple2<String, Integer>> {
+	public static class MySourceMap extends RichMapFunction<String, Tuple3<Long, String, Integer>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -175,44 +186,55 @@ public class WindowJoin {
 		}
 
 		@Override
-		public Tuple2<String, Integer> map(String line) throws Exception {
+		public Tuple3<Long, String, Integer> map(String line) throws Exception {
 			record = line.substring(1, line.length() - 1).split(",");
-			return new Tuple2<String, Integer>(record[0], Integer.parseInt(record[1]));
+			return new Tuple3<>(Long.parseLong(record[0]), record[1], Integer.parseInt(record[2]));
 		}
 	}
 
 	public static class MyJoinFunction
 			implements
-			JoinFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> {
+			JoinFunction<Tuple3<Long, String, Integer>, Tuple3<Long, String, Integer>, Tuple3<String, Integer, Integer>> {
 
 		private static final long serialVersionUID = 1L;
 
-		private Tuple3<String, Integer, Integer> joined = new Tuple3<String, Integer, Integer>();
+		private Tuple3<String, Integer, Integer> joined = new Tuple3<>();
 
 		@Override
-		public Tuple3<String, Integer, Integer> join(Tuple2<String, Integer> first,
-				Tuple2<String, Integer> second) throws Exception {
-			joined.f0 = first.f0;
-			joined.f1 = first.f1;
-			joined.f2 = second.f1;
+		public Tuple3<String, Integer, Integer> join(Tuple3<Long, String, Integer> first,
+				Tuple3<Long, String, Integer> second) throws Exception {
+			joined.f0 = first.f1;
+			joined.f1 = first.f2;
+			joined.f2 = second.f2;
 			return joined;
 		}
 	}
 
-	public static class MyTimestamp implements Timestamp<Tuple2<String, Integer>> {
-
+	private static class MyTimestampExtractor implements TimestampExtractor<Tuple3<Long, String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
-		private int counter;
+		@Override
+		public long extractTimestamp(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+			return element.f0;
+		}
 
-		public MyTimestamp(int starttime) {
-			this.counter = starttime;
+		@Override
+		public long emitWatermark(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+			return element.f0 - 1;
 		}
 
 		@Override
-		public long getTimestamp(Tuple2<String, Integer> value) {
-			counter += SLEEP_TIME;
-			return counter;
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class NameKeySelector implements KeySelector<Tuple3<Long, String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple3<Long, String, Integer> value) throws Exception {
+			return value.f1;
 		}
 	}
 
@@ -253,7 +275,12 @@ public class WindowJoin {
 		return true;
 	}
 
-	private static void setInputStreams(StreamExecutionEnvironment env) {
+	private static Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> getInputStreams(
+			StreamExecutionEnvironment env) {
+
+		DataStream<Tuple3<Long, String, Integer>> grades;
+		DataStream<Tuple3<Long, String, Integer>> salaries;
+
 		if (fileInput) {
 			grades = env.readTextFile(gradesPath).map(new MySourceMap());
 			salaries = env.readTextFile(salariesPath).map(new MySourceMap());
@@ -261,5 +288,8 @@ public class WindowJoin {
 			grades = env.addSource(new GradeSource());
 			salaries = env.addSource(new SalarySource());
 		}
+
+		return Tuple2.of(grades, salaries);
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
index 23d29b1..15c1280 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
@@ -19,42 +19,42 @@ package org.apache.flink.streaming.examples.join.util;
 
 public class WindowJoinData {
 
-	public static final String GRADES_INPUT = "(john,5)\n" + "(tom,3)\n" + "(alice,1)\n" + "(grace,5)\n" +
-			"(john,4)\n" + "(bob,1)\n" + "(alice,2)\n" + "(alice,3)\n" + "(bob,5)\n" + "(alice,3)\n" + "(tom,5)\n" +
-			"(john,2)\n" + "(john,1)\n" + "(grace,2)\n" + "(jerry,2)\n" + "(tom,4)\n" + "(bob,4)\n" + "(bob,2)\n" +
-			"(tom,2)\n" + "(alice,5)\n" + "(grace,5)\n" + "(grace,1)\n" + "(alice,1)\n" + "(grace,3)\n" + "(tom,1)\n" +
-			"(jerry,5)\n" + "(john,3)\n" + "(john,4)\n" + "(john,1)\n" + "(jerry,3)\n" + "(grace,3)\n" + "(bob,3)\n" +
-			"(john,3)\n" + "(jerry,4)\n" + "(tom,5)\n" + "(tom,4)\n" + "(john,2)\n" + "(jerry,1)\n" + "(bob,1)\n" +
-			"(john,5)\n" + "(grace,4)\n" + "(tom,5)\n" + "(john,4)\n" + "(tom,1)\n" + "(grace,1)\n" + "(john,2)\n" +
-			"(jerry,3)\n" + "(jerry,5)\n" + "(tom,2)\n" + "(tom,2)\n" + "(alice,4)\n" + "(tom,4)\n" + "(jerry,4)\n" +
-			"(john,3)\n" + "(grace,4)\n" + "(tom,3)\n" + "(jerry,4)\n" + "(john,5)\n" + "(john,4)\n" + "(jerry,1)\n" +
-			"(john,5)\n" + "(alice,2)\n" + "(tom,1)\n" + "(alice,5)\n" + "(grace,4)\n" + "(bob,4)\n" + "(jerry,1)\n" +
-			"(john,5)\n" + "(tom,4)\n" + "(tom,5)\n" + "(jerry,5)\n" + "(tom,1)\n" + "(grace,3)\n" + "(bob,5)\n" +
-			"(john,1)\n" + "(alice,1)\n" + "(grace,3)\n" + "(grace,1)\n" + "(jerry,1)\n" + "(jerry,4)\n" +
-			"(bob,4)\n" + "(alice,3)\n" + "(tom,5)\n" + "(alice,4)\n" + "(alice,4)\n" + "(grace,4)\n" + "(john,5)\n" +
-			"(john,5)\n" + "(grace,4)\n" + "(tom,4)\n" + "(john,4)\n" + "(john,5)\n" + "(alice,5)\n" + "(jerry,5)\n" +
-			"(john,3)\n" + "(tom,5)\n" + "(jerry,4)\n" + "(grace,4)\n" + "(john,3)\n" + "(bob,2)";
+	public static final String GRADES_INPUT = "(0,john,5)\n" + "(0,tom,3)\n" + "(0,alice,1)\n" + "(0,grace,5)\n" +
+			"(1,john,4)\n" + "(1,bob,1)\n" + "(1,alice,2)\n" + "(1,alice,3)\n" + "(1,bob,5)\n" + "(1,alice,3)\n" + "(1,tom,5)\n" +
+			"(2,john,2)\n" + "(2,john,1)\n" + "(2,grace,2)\n" + "(2,jerry,2)\n" + "(2,tom,4)\n" + "(2,bob,4)\n" + "(2,bob,2)\n" +
+			"(3, tom,2)\n" + "(3,alice,5)\n" + "(3,grace,5)\n" + "(3,grace,1)\n" + "(3,alice,1)\n" + "(3,grace,3)\n" + "(3,tom,1)\n" +
+			"(4,jerry,5)\n" + "(4,john,3)\n" + "(4,john,4)\n" + "(4,john,1)\n" + "(4,jerry,3)\n" + "(4,grace,3)\n" + "(4,bob,3)\n" +
+			"(5,john,3)\n" + "(5,jerry,4)\n" + "(5,tom,5)\n" + "(5,tom,4)\n" + "(5,john,2)\n" + "(5,jerry,1)\n" + "(5,bob,1)\n" +
+			"(6,john,5)\n" + "(6,grace,4)\n" + "(6,tom,5)\n" + "(6,john,4)\n" + "(6,tom,1)\n" + "(6,grace,1)\n" + "(6,john,2)\n" +
+			"(7,jerry,3)\n" + "(7,jerry,5)\n" + "(7,tom,2)\n" + "(7,tom,2)\n" + "(7,alice,4)\n" + "(7,tom,4)\n" + "(7,jerry,4)\n" +
+			"(8,john,3)\n" + "(8,grace,4)\n" + "(8,tom,3)\n" + "(8,jerry,4)\n" + "(8,john,5)\n" + "(8,john,4)\n" + "(8,jerry,1)\n" +
+			"(9,john,5)\n" + "(9,alice,2)\n" + "(9,tom,1)\n" + "(9,alice,5)\n" + "(9,grace,4)\n" + "(9,bob,4)\n" + "(9,jerry,1)\n" +
+			"(10,john,5)\n" + "(10,tom,4)\n" + "(10,tom,5)\n" + "(10,jerry,5)\n" + "(10,tom,1)\n" + "(10,grace,3)\n" + "(10,bob,5)\n" +
+			"(11,john,1)\n" + "(11,alice,1)\n" + "(11,grace,3)\n" + "(11,grace,1)\n" + "(11,jerry,1)\n" + "(11,jerry,4)\n" +
+			"(12,bob,4)\n" + "(12,alice,3)\n" + "(12,tom,5)\n" + "(12,alice,4)\n" + "(12,alice,4)\n" + "(12,grace,4)\n" + "(12,john,5)\n" +
+			"(13,john,5)\n" + "(13,grace,4)\n" + "(13,tom,4)\n" + "(13,john,4)\n" + "(13,john,5)\n" + "(13,alice,5)\n" + "(13,jerry,5)\n" +
+			"(14,john,3)\n" + "(14,tom,5)\n" + "(14,jerry,4)\n" + "(14,grace,4)\n" + "(14,john,3)\n" + "(14,bob,2)";
 
-	public static final String SALARIES_INPUT = "(john,6469)\n" + "(jerry,6760)\n" + "(jerry,8069)\n" +
-			"(tom,3662)\n" + "(grace,8427)\n" + "(john,9425)\n" + "(bob,9018)\n" + "(john,352)\n" + "(tom,3770)\n" +
-			"(grace,7622)\n" + "(jerry,7441)\n" + "(alice,1468)\n" + "(bob,5472)\n" + "(grace,898)\n" +
-			"(tom,3849)\n" + "(grace,1865)\n" + "(alice,5582)\n" + "(john,9511)\n" + "(alice,1541)\n" +
-			"(john,2477)\n" + "(grace,3561)\n" + "(john,1670)\n" + "(grace,7290)\n" + "(grace,6565)\n" +
-			"(tom,6179)\n" + "(tom,1601)\n" + "(john,2940)\n" + "(bob,4685)\n" + "(bob,710)\n" + "(bob,5936)\n" +
-			"(jerry,1412)\n" + "(grace,6515)\n" + "(grace,3321)\n" + "(tom,8088)\n" + "(john,2876)\n" +
-			"(bob,9896)\n" + "(grace,7368)\n" + "(grace,9749)\n" + "(bob,2048)\n" + "(alice,4782)\n" +
-			"(alice,3375)\n" + "(tom,5841)\n" + "(bob,958)\n" + "(bob,5258)\n" + "(tom,3935)\n" + "(jerry,4394)\n" +
-			"(alice,102)\n" + "(alice,4931)\n" + "(alice,5240)\n" + "(jerry,7951)\n" + "(john,5675)\n" +
-			"(bob,609)\n" + "(alice,5997)\n" + "(jerry,9651)\n" + "(alice,1328)\n" + "(bob,1022)\n" +
-			"(grace,2578)\n" + "(jerry,9704)\n" + "(tom,4476)\n" + "(grace,3784)\n" + "(alice,6144)\n" +
-			"(bob,6213)\n" + "(alice,7525)\n" + "(jerry,2908)\n" + "(grace,8464)\n" + "(jerry,9920)\n" +
-			"(bob,3720)\n" + "(bob,7612)\n" + "(alice,7211)\n" + "(jerry,6484)\n" + "(alice,1711)\n" +
-			"(jerry,5994)\n" + "(grace,928)\n" + "(jerry,2492)\n" + "(grace,9080)\n" + "(tom,4330)\n" +
-			"(bob,8302)\n" + "(john,4981)\n" + "(tom,1781)\n" + "(grace,1379)\n" + "(jerry,3700)\n" +
-			"(jerry,3584)\n" + "(jerry,2038)\n" + "(jerry,3902)\n" + "(tom,1336)\n" + "(jerry,7500)\n" +
-			"(tom,3648)\n" + "(alice,2533)\n" + "(tom,8685)\n" + "(bob,3968)\n" + "(tom,3241)\n" + "(bob,7461)\n" +
-			"(jerry,2138)\n" + "(alice,7503)\n" + "(alice,6424)\n" + "(tom,140)\n" + "(john,9802)\n" +
-			"(grace,2977)\n" + "(grace,889)\n" + "(john,1338)";
+	public static final String SALARIES_INPUT = "(0,john,6469)\n" + "(0,jerry,6760)\n" + "(0,jerry,8069)\n" +
+			"(1,tom,3662)\n" + "(1,grace,8427)\n" + "(1,john,9425)\n" + "(1,bob,9018)\n" + "(1,john,352)\n" + "(1,tom,3770)\n" +
+			"(2,grace,7622)\n" + "(2,jerry,7441)\n" + "(2,alice,1468)\n" + "(2,bob,5472)\n" + "(2,grace,898)\n" +
+			"(3,tom,3849)\n" + "(3,grace,1865)\n" + "(3,alice,5582)\n" + "(3,john,9511)\n" + "(3,alice,1541)\n" +
+			"(4,john,2477)\n" + "(4,grace,3561)\n" + "(4,john,1670)\n" + "(4,grace,7290)\n" + "(4,grace,6565)\n" +
+			"(5,tom,6179)\n" + "(5,tom,1601)\n" + "(5,john,2940)\n" + "(5,bob,4685)\n" + "(5,bob,710)\n" + "(5,bob,5936)\n" +
+			"(6,jerry,1412)\n" + "(6,grace,6515)\n" + "(6,grace,3321)\n" + "(6,tom,8088)\n" + "(6,john,2876)\n" +
+			"(7,bob,9896)\n" + "(7,grace,7368)\n" + "(7,grace,9749)\n" + "(7,bob,2048)\n" + "(7,alice,4782)\n" +
+			"(8,alice,3375)\n" + "(8,tom,5841)\n" + "(8,bob,958)\n" + "(8,bob,5258)\n" + "(8,tom,3935)\n" + "(8,jerry,4394)\n" +
+			"(9,alice,102)\n" + "(9,alice,4931)\n" + "(9,alice,5240)\n" + "(9,jerry,7951)\n" + "(9,john,5675)\n" +
+			"(10,bob,609)\n" + "(10,alice,5997)\n" + "(10,jerry,9651)\n" + "(10,alice,1328)\n" + "(10,bob,1022)\n" +
+			"(11,grace,2578)\n" + "(11,jerry,9704)\n" + "(11,tom,4476)\n" + "(11,grace,3784)\n" + "(11,alice,6144)\n" +
+			"(12,bob,6213)\n" + "(12,alice,7525)\n" + "(12,jerry,2908)\n" + "(12,grace,8464)\n" + "(12,jerry,9920)\n" +
+			"(13,bob,3720)\n" + "(13,bob,7612)\n" + "(13,alice,7211)\n" + "(13,jerry,6484)\n" + "(13,alice,1711)\n" +
+			"(14,jerry,5994)\n" + "(14,grace,928)\n" + "(14,jerry,2492)\n" + "(14,grace,9080)\n" + "(14,tom,4330)\n" +
+			"(15,bob,8302)\n" + "(15,john,4981)\n" + "(15,tom,1781)\n" + "(15,grace,1379)\n" + "(15,jerry,3700)\n" +
+			"(16,jerry,3584)\n" + "(16,jerry,2038)\n" + "(16,jerry,3902)\n" + "(16,tom,1336)\n" + "(16,jerry,7500)\n" +
+			"(17,tom,3648)\n" + "(17,alice,2533)\n" + "(17,tom,8685)\n" + "(17,bob,3968)\n" + "(17,tom,3241)\n" + "(17,bob,7461)\n" +
+			"(18,jerry,2138)\n" + "(18,alice,7503)\n" + "(18,alice,6424)\n" + "(18,tom,140)\n" + "(18,john,9802)\n" +
+			"(19,grace,2977)\n" + "(19,grace,889)\n" + "(19,john,1338)";
 
 	private WindowJoinData() {
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
index 239f1fa..225dab7 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.scala.examples.join
 import java.util.concurrent.TimeUnit
 
 import org.apache.flink.streaming.api.scala._
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
 
 import scala.Stream._
 import scala.language.postfixOps
@@ -32,8 +34,8 @@ object WindowJoin {
   // PROGRAM
   // *************************************************************************
 
-  case class Grade(name: String, grade: Int)
-  case class Salary(name: String, salary: Int)
+  case class Grade(time: Long, name: String, grade: Int)
+  case class Salary(time: Long, name: String, salary: Int)
   case class Person(name: String, grade: Int, salary: Int)
 
   def main(args: Array[String]) {
@@ -43,6 +45,7 @@ object WindowJoin {
     }
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.getConfig.enableTimestamps()
 
     //Create streams for grades and salaries by mapping the inputs to the corresponding objects
     val grades = setGradesInput(env)
@@ -50,11 +53,11 @@ object WindowJoin {
 
     //Join the two input streams by name on the last 2 seconds every second and create new
     //Person objects containing both grade and salary
-    val joined =
-      grades.join(salaries).onWindow(2, TimeUnit.SECONDS)
-                      .every(1, TimeUnit.SECONDS)
-                      .where("name")
-                      .equalTo("name") { (g, s) => Person(g.name, g.grade, s.salary) }
+    val joined = grades.join(salaries)
+        .where(_.name)
+        .equalTo(_.name)
+        .window(SlidingTimeWindows.of(Time.of(2, TimeUnit.SECONDS), Time.of(1, TimeUnit.SECONDS)))
+        .apply { (g, s) => Person(g.name, g.grade, s.salary) }
 
     if (fileOutput) {
       joined.writeAsText(outputPath)
@@ -74,27 +77,27 @@ object WindowJoin {
   val salaryMax = 10000
   val sleepInterval = 100
   
-  def gradeStream(): Stream[(String, Int)] = {
-    def gradeMapper(names: Array[String])(x: Int): (String, Int) =
+  def gradeStream: Stream[(Long, String, Int)] = {
+    def gradeMapper(names: Array[String])(x: Int): (Long, String, Int) =
       {
         if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
-        (names(Random.nextInt(names.length)), Random.nextInt(gradeCount))
+        (System.currentTimeMillis(),names(Random.nextInt(names.length)),Random.nextInt(gradeCount))
       }
     range(1, 100).map(gradeMapper(names))
   }
 
-  def salaryStream(): Stream[(String, Int)] = {
-    def salaryMapper(x: Int): (String, Int) =
+  def salaryStream: Stream[(Long, String, Int)] = {
+    def salaryMapper(x: Int): (Long, String, Int) =
       {
         if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
-        (names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
+        (System.currentTimeMillis(), names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
       }
     range(1, 100).map(salaryMapper)
   }
 
-  def parseMap(line : String): (String, Int) = {
+  def parseMap(line : String): (Long, String, Int) = {
     val record = line.substring(1, line.length - 1).split(",")
-    (record(0), record(1).toInt)
+    (record(0).toLong, record(1), record(2).toInt)
   }
 
   // *************************************************************************
@@ -130,23 +133,23 @@ object WindowJoin {
       System.out.println("  Provide parameter to write to file.")
       System.out.println("  Usage: WindowJoin <result path>")
     }
-    return true
+    true
   }
 
   private def setGradesInput(env: StreamExecutionEnvironment) : DataStream[Grade] = {
     if (fileInput) {
-      env.readTextFile(gradesPath).map(parseMap(_)).map(x => Grade(x._1, x._2))
+      env.readTextFile(gradesPath).map(parseMap _ ).map(x => Grade(x._1, x._2, x._3))
     } else {
-      env.fromCollection(gradeStream).map(x => Grade(x._1, x._2))
+      env.fromCollection(gradeStream).map(x => Grade(x._1, x._2, x._3))
     }
   }
 
   private def setSalariesInput(env: StreamExecutionEnvironment) : DataStream[Salary] = {
     if (fileInput) {
-      env.readTextFile(salariesPath).map(parseMap(_)).map(x => Salary(x._1, x._2))
+      env.readTextFile(salariesPath).map(parseMap _).map(x => Salary(x._1, x._2, x._3))
     }
     else {
-      env.fromCollection(salaryStream).map(x => Salary(x._1, x._2))
+      env.fromCollection(salaryStream).map(x => Salary(x._1, x._2, x._3))
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 aae4b93..e657b67 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,51 +1,50 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-// 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});
-//	}
-//}
+/*
+ * 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});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 0aa884f..08ce890 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,51 +1,50 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *     http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-// 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});
-//	}
-//}
+/*
+ * 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});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
new file mode 100644
index 0000000..1b16e44
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
@@ -0,0 +1,294 @@
+/*
+ * 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.CoGroupFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.datastream.{CoGroupedStreams => JavaCoGroupedStreams}
+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.util.Collector
+
+import scala.collection.JavaConverters._
+
+import scala.reflect.ClassTag
+
+/**
+ * `CoGroupedStreams` represents two [[DataStream]]s that have been co-grouped.
+ * A streaming co-group operation is evaluated over elements in a window.
+ *
+ * To finalize the co-group operation you also need to specify a [[KeySelector]] for
+ * both the first and second input and a [[WindowAssigner]]
+ *
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * Example:
+ *
+ * {{{
+ * val one: DataStream[(String, Int)]  = ...
+ * val two: DataStream[(String, Int)] = ...
+ *
+ * val result = one.coGroup(two)
+ *     .where(new MyFirstKeySelector())
+ *     .equalTo(new MyFirstKeySelector())
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyCoGroupFunction())
+ * } }}}
+ */
+object CoGroupedStreams {
+
+  /**
+   * A co-group operation that does not yet have its [[KeySelector]]s defined.
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   */
+  class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the first input.
+     */
+    def where[KEY](keySelector: T1 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T1, KEY] {
+        def getKey(in: T1) = cleanFun(in)
+      }
+      new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, null)
+    }
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the second input.
+     */
+    def equalTo[KEY](keySelector: T2 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T2, KEY] {
+        def getKey(in: T2) = cleanFun(in)
+      }
+      new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, null, javaSelector)
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+  /**
+   * A co-group operation that has [[KeySelector]]s defined for either both or
+   * one input.
+   *
+   * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
+   * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   * @tparam KEY Type of the key. This must be the same for both inputs
+   */
+  class WithKey[T1, T2, KEY](
+      input1: DataStream[T1],
+      input2: DataStream[T2],
+      keySelector1: KeySelector[T1, KEY],
+      keySelector2: KeySelector[T2, KEY]) {
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the first input.
+     */
+    def where(keySelector: T1 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T1, KEY] {
+        def getKey(in: T1) = cleanFun(in)
+      }
+      new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2)
+    }
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the second input.
+     */
+    def equalTo(keySelector: T2 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T2, KEY] {
+        def getKey(in: T2) = cleanFun(in)
+      }
+      new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector)
+    }
+
+    /**
+     * Specifies the window on which the co-group operation works.
+     */
+    def window[W <: Window](
+        assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
+        : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+      if (keySelector1 == null || keySelector2 == null) {
+        throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
+          "inputs using where() and equalTo().")
+      }
+      new CoGroupedStreams.WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        clean(assigner),
+        null,
+        null)
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+  /**
+   * A co-group operation that has [[KeySelector]]s defined for both inputs as
+   * well as a [[WindowAssigner]].
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   * @tparam KEY Type of the key. This must be the same for both inputs
+   * @tparam W Type of { @link Window} on which the co-group operation works.
+   */
+  class WithWindow[T1, T2, KEY, W <: Window](
+      input1: DataStream[T1],
+      input2: DataStream[T2],
+      keySelector1: KeySelector[T1, KEY],
+      keySelector2: KeySelector[T2, KEY],
+      windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
+      trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
+      evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
+
+
+    /**
+     * Sets the [[Trigger]] that should be used to trigger window emission.
+     */
+    def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+    : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+      new WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        windowAssigner,
+        newTrigger,
+        evictor)
+    }
+
+    /**
+     * 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(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+    : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+      new WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        windowAssigner,
+        trigger,
+        newEvictor)
+    }
+
+    /**
+     * Completes the co-group operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[O: TypeInformation: ClassTag](
+        fun: (Iterator[T1], Iterator[T2]) => O): DataStream[O] = {
+      require(fun != null, "CoGroup function must not be null.")
+
+      val coGrouper = new CoGroupFunction[T1, T2, O] {
+        val cleanFun = clean(fun)
+        def coGroup(
+            left: java.lang.Iterable[T1],
+            right: java.lang.Iterable[T2], out: Collector[O]) = {
+          out.collect(cleanFun(left.iterator().asScala, right.iterator().asScala))
+        }
+      }
+      apply(coGrouper)
+    }
+
+    /**
+     * Completes the co-group operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[O: TypeInformation: ClassTag](
+        fun: (Iterator[T1], Iterator[T2], Collector[O]) => Unit): DataStream[O] = {
+      require(fun != null, "CoGroup function must not be null.")
+
+      val coGrouper = new CoGroupFunction[T1, T2, O] {
+        val cleanFun = clean(fun)
+        def coGroup(
+            left: java.lang.Iterable[T1],
+            right: java.lang.Iterable[T2], out: Collector[O]) = {
+          cleanFun(left.iterator.asScala, right.iterator.asScala, out)
+        }
+      }
+      apply(coGrouper)
+    }
+
+    /**
+     * Completes the co-group operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[T: TypeInformation](function: CoGroupFunction[T1, T2, T]): DataStream[T] = {
+
+      val coGroup = JavaCoGroupedStreams.createCoGroup(input1.getJavaStream, input2.getJavaStream)
+
+      coGroup
+        .where(keySelector1)
+        .equalTo(keySelector2)
+        .window(windowAssigner)
+        .trigger(trigger)
+        .evictor(evictor)
+        .apply(clean(function), implicitly[TypeInformation[T]])
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+
+  /**
+   * Creates a new co-group operation from the two given inputs.
+   */
+  def createCoGroup[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
+      : CoGroupedStreams.Unspecified[T1, T2] = {
+    new CoGroupedStreams.Unspecified[T1, T2](input1, input2)
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 8aeacb4..7babc40 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
@@ -751,18 +751,20 @@ class DataStream[T](javaStream: JavaStream[T]) {
   }
 
   /**
-   * Initiates a temporal Join transformation that joins the elements of two
-   * data streams on key equality over a specified time window.
-   *
-   * 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.
-   *
+   * Creates a co-group operation. See [[CoGroupedStreams]] for an example of how the keys
+   * and window can be specified.
    */
-  def join[R](stream: DataStream[R]): StreamJoinOperator[T, R] =
-    new StreamJoinOperator[T, R](javaStream, stream.getJavaStream)
+  def coGroup[T2](otherStream: DataStream[T2]): CoGroupedStreams.Unspecified[T, T2] = {
+    CoGroupedStreams.createCoGroup(this, otherStream)
+  }
+
+  /**
+   * Creates a join operation. See [[JoinedStreams]] for an example of how the keys
+   * and window can be specified.
+   */
+  def join[T2](otherStream: DataStream[T2]): JoinedStreams.Unspecified[T, T2] = {
+    JoinedStreams.createJoin(this, otherStream)
+  }
 
   /**
    * Writes a DataStream to the standard output stream (stdout). For each

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
new file mode 100644
index 0000000..be059b8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
@@ -0,0 +1,303 @@
+/*
+ * 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.{FlatJoinFunction, JoinFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.streaming.api.datastream.{JoinedStreams => JavaJoinedStreams, CoGroupedStreams => JavaCoGroupedStreams}
+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.util.Collector
+
+import scala.reflect.ClassTag
+
+/**
+ * `JoinedStreams` represents two [[DataStream]]s that have been joined.
+ * A streaming join operation is evaluated over elements in a window.
+ *
+ * To finalize the join operation you also need to specify a [[KeySelector]] for
+ * both the first and second input and a [[WindowAssigner]]
+ *
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * Example:
+ *
+ * {{{
+ * val one: DataStream[(String, Int)]  = ...
+ * val two: DataStream[(String, Int)] = ...
+ *
+ * val result = one.join(two)
+ *     .where {t => ... }
+ *     .equal {t => ... }
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyJoinFunction())
+ * } }}}
+ */
+object JoinedStreams {
+
+  /**
+   * A join operation that does not yet have its [[KeySelector]]s defined.
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   */
+  class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the first input.
+     */
+    def where[KEY](keySelector: T1 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T1, KEY] {
+        def getKey(in: T1) = cleanFun(in)
+      }
+      new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, null)
+    }
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the second input.
+     */
+    def equalTo[KEY](keySelector: T2 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T2, KEY] {
+        def getKey(in: T2) = cleanFun(in)
+      }
+      new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, null, javaSelector)
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+  /**
+   * A join operation that has [[KeySelector]]s defined for either both or
+   * one input.
+   *
+   * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
+   * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   * @tparam KEY Type of the key. This must be the same for both inputs
+   */
+  class WithKey[T1, T2, KEY](
+      input1: DataStream[T1],
+      input2: DataStream[T2],
+      keySelector1: KeySelector[T1, KEY],
+      keySelector2: KeySelector[T2, KEY]) {
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the first input.
+     */
+    def where(keySelector: T1 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T1, KEY] {
+        def getKey(in: T1) = cleanFun(in)
+      }
+      new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2)
+    }
+
+    /**
+     * Specifies a [[KeySelector]] for elements from the second input.
+     */
+    def equalTo(keySelector: T2 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+      val cleanFun = clean(keySelector)
+      val javaSelector = new KeySelector[T2, KEY] {
+        def getKey(in: T2) = cleanFun(in)
+      }
+      new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector)
+    }
+
+    /**
+     * Specifies the window on which the join operation works.
+     */
+    def window[W <: Window](
+        assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
+        : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+      if (keySelector1 == null || keySelector2 == null) {
+        throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
+          "inputs using where() and equalTo().")
+      }
+      new JoinedStreams.WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        clean(assigner),
+        null,
+        null)
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+  /**
+   * A join operation that has [[KeySelector]]s defined for both inputs as
+   * well as a [[WindowAssigner]].
+   *
+   * @tparam T1 Type of the elements from the first input
+   * @tparam T2 Type of the elements from the second input
+   * @tparam KEY Type of the key. This must be the same for both inputs
+   * @tparam W Type of { @link Window} on which the join operation works.
+   */
+  class WithWindow[T1, T2, KEY, W <: Window](
+      input1: DataStream[T1],
+      input2: DataStream[T2],
+      keySelector1: KeySelector[T1, KEY],
+      keySelector2: KeySelector[T2, KEY],
+      windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
+      trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
+      evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
+
+
+    /**
+     * Sets the [[Trigger]] that should be used to trigger window emission.
+     */
+    def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+    : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+      new WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        windowAssigner,
+        newTrigger,
+        evictor)
+    }
+
+    /**
+     * 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(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+    : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+      new WithWindow[T1, T2, KEY, W](
+        input1,
+        input2,
+        keySelector1,
+        keySelector2,
+        windowAssigner,
+        trigger,
+        newEvictor)
+    }
+
+    /**
+     * Completes the join operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[O: TypeInformation: ClassTag](fun: (T1, T2) => O): DataStream[O] = {
+      require(fun != null, "Join function must not be null.")
+
+      val joiner = new FlatJoinFunction[T1, T2, O] {
+        val cleanFun = clean(fun)
+        def join(left: T1, right: T2, out: Collector[O]) = {
+          out.collect(cleanFun(left, right))
+        }
+      }
+      apply(joiner)
+    }
+
+    /**
+     * Completes the join operation with the user function that is executed
+     * for windowed groups.
+     */
+
+    def apply[O: TypeInformation: ClassTag](fun: (T1, T2, Collector[O]) => Unit): DataStream[O] = {
+      require(fun != null, "Join function must not be null.")
+
+      val joiner = new FlatJoinFunction[T1, T2, O] {
+        val cleanFun = clean(fun)
+        def join(left: T1, right: T2, out: Collector[O]) = {
+          cleanFun(left, right, out)
+        }
+      }
+      apply(joiner)
+    }
+
+    /**
+     * Completes the join operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[T: TypeInformation](function: JoinFunction[T1, T2, T]): DataStream[T] = {
+
+      val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+
+      join
+        .where(keySelector1)
+        .equalTo(keySelector2)
+        .window(windowAssigner)
+        .trigger(trigger)
+        .evictor(evictor)
+        .apply(clean(function), implicitly[TypeInformation[T]])
+    }
+
+    /**
+     * Completes the join operation with the user function that is executed
+     * for windowed groups.
+     */
+    def apply[T: TypeInformation](function: FlatJoinFunction[T1, T2, T]): DataStream[T] = {
+
+      val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+
+      join
+        .where(keySelector1)
+        .equalTo(keySelector2)
+        .window(windowAssigner)
+        .trigger(trigger)
+        .evictor(evictor)
+        .apply(clean(function), implicitly[TypeInformation[T]])
+    }
+
+    /**
+     * 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(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+    }
+  }
+
+
+  /**
+   * Creates a new join operation from the two given inputs.
+   */
+  def createJoin[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
+      : JoinedStreams.Unspecified[T1, T2] = {
+    new JoinedStreams.Unspecified[T1, T2](input1, input2)
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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
deleted file mode 100644
index e0bbaf8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
+++ /dev/null
@@ -1,203 +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.JoinFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-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.util.keys.KeySelectorUtil
-
-import scala.Array.canBuildFrom
-import scala.reflect.ClassTag
-
-class StreamJoinOperator[I1, I2](i1: JavaStream[I1], i2: JavaStream[I2]) extends 
-TemporalOperator[I1, I2, StreamJoinOperator.JoinWindow[I1, I2]](i1, i2) {
-
-  override def createNextWindowOperator() = {
-    new StreamJoinOperator.JoinWindow[I1, I2](this)
-  }
-}
-
-object StreamJoinOperator {
-
-  class JoinWindow[I1, I2](private[flink] val op: StreamJoinOperator[I1, I2]) extends
-  TemporalWindow[JoinWindow[I1, I2]] {
-
-    private[flink] val type1 = op.input1.getType()
-
-    /**
-     * Continues a temporal Join transformation by defining
-     * the fields in the first stream to be used as keys for the join.
-     * The resulting incomplete join can be completed by JoinPredicate.equalTo()
-     * to define the second key.
-     */
-    def where(fields: Int*) = {
-      new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys(
-        new Keys.ExpressionKeys(fields.toArray, type1),
-        type1,
-        op.input1.getExecutionEnvironment.getConfig))
-    }
-
-    /**
-     * Continues a temporal Join transformation by defining
-     * the fields in the first stream to be used as keys for the join.
-     * The resulting incomplete join can be completed by JoinPredicate.equalTo()
-     * to define the second key.
-     */
-    def where(firstField: String, otherFields: String*) =
-      new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys(
-        new Keys.ExpressionKeys(firstField +: otherFields.toArray, type1),
-        type1,
-        op.input1.getExecutionEnvironment.getConfig))
-
-    /**
-     * Continues a temporal Join transformation by defining
-     * the keyselector function that will be used to extract keys from the first stream
-     * for the join.
-     * The resulting incomplete join can be completed by JoinPredicate.equalTo()
-     * to define the second key.
-     */
-    def where[K: TypeInformation](fun: (I1) => K) = {
-      val keyType = implicitly[TypeInformation[K]]
-      val cleanFun = op.input1.clean(fun)
-      val keyExtractor = new KeySelector[I1, K] {
-        def getKey(in: I1) = cleanFun(in)
-      }
-      new JoinPredicate[I1, I2](op, keyExtractor)
-    }
-
-    override def every(length: Long, timeUnit: TimeUnit): JoinWindow[I1, I2] = {
-      every(timeUnit.toMillis(length))
-    }
-
-    override def every(length: Long): JoinWindow[I1, I2] = {
-      op.slideInterval = length
-      this
-    }
-
-  }
-
-  class JoinPredicate[I1, I2](private[flink] val op: StreamJoinOperator[I1, I2],
-    private[flink] val keys1: KeySelector[I1, _]) {
-    private[flink] var keys2: KeySelector[I2, _] = null
-    private[flink] val type2 = op.input2.getType()
-
-    /**
-     * Creates a temporal join transformation by defining the second join key.
-     * The returned transformation wrapes each joined element pair in a tuple2:
-     * (first, second)
-     * To define a custom wrapping, use JoinedStream.apply(...)
-     */
-    def equalTo(fields: Int*): JoinedStream[I1, I2] = {
-      finish(KeySelectorUtil.getSelectorForKeys(
-        new Keys.ExpressionKeys(fields.toArray, type2),
-        type2,
-        op.input1.getExecutionEnvironment.getConfig))
-    }
-
-    /**
-     * Creates a temporal join transformation by defining the second join key.
-     * The returned transformation wrapes each joined element pair in a tuple2:
-     * (first, second)
-     * To define a custom wrapping, use JoinedStream.apply(...)
-     */
-    def equalTo(firstField: String, otherFields: String*): JoinedStream[I1, I2] =
-      finish(KeySelectorUtil.getSelectorForKeys(
-        new Keys.ExpressionKeys(firstField +: otherFields.toArray, type2),
-        type2,
-        op.input1.getExecutionEnvironment.getConfig))
-
-    /**
-     * Creates a temporal join transformation by defining the second join key.
-     * The returned transformation wrapes each joined element pair in a tuple2:
-     * (first, second)
-     * To define a custom wrapping, use JoinedStream.apply(...)
-     */
-    def equalTo[K: TypeInformation](fun: (I2) => K): JoinedStream[I1, I2] = {
-      val keyType = implicitly[TypeInformation[K]]
-      val cleanFun = op.input1.clean(fun)
-      val keyExtractor = new KeySelector[I2, K] {
-        def getKey(in: I2) = cleanFun(in)
-      }
-      finish(keyExtractor)
-    }
-
-    private def finish(keys2: KeySelector[I2, _]): JoinedStream[I1, I2] = {
-      this.keys2 = keys2
-      new JoinedStream[I1, I2](this, createJoinOperator())
-    }
-
-    private def createJoinOperator(): JavaStream[(I1, I2)] = {
-
-//      val returnType = createTuple2TypeInformation[I1, I2](op.input1.getType, op.input2.getType)
-//      op.input1.keyBy(keys1).connect(op.input2.keyBy(keys2))
-//        .addGeneralWindowCombine(getJoinWindowFunction(this, (_, _)),
-//          returnType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)
-      null
-    }
-  }
-
-  class JoinedStream[I1, I2](
-      jp: JoinPredicate[I1, I2],
-      javaStream: JavaStream[(I1, I2)]) extends DataStream[(I1, I2)](javaStream) {
-
-    private val op = jp.op
-
-    /**
-     * Sets a wrapper for the joined elements. For each joined pair, the result of the
-     * udf call will be emitted.
-     */
-    def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = {
-
-      val cleanFun = clean(getJoinWindowFunction(jp, fun))
-
-//      op.input1.keyBy(jp.keys1).connect(op.input2.keyBy(jp.keys2))
-//        .addGeneralWindowCombine[R](
-//          cleanFun,
-//          implicitly[TypeInformation[R]],
-//          op.windowSize,
-//          op.slideInterval,
-//          op.timeStamp1,
-//          op.timeStamp2)
-      null
-    }
-  }
-
-  private[flink] def getJoinWindowFunction[I1, I2, R](jp: JoinPredicate[I1, I2],
-    joinFunction: (I1, I2) => R) = {
-    require(joinFunction != null, "Join function must not be null.")
-
-    val cleanFun = jp.op.input1.clean(joinFunction)
-
-    val joinFun = new JoinFunction[I1, I2, R] {
-      override def join(first: I1, second: I2): R = {
-        cleanFun(first, second)
-      }
-    }
-
-//    new JoinWindowFunction[I1, I2, R](jp.keys1, jp.keys2, joinFun)
-    null
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
deleted file mode 100644
index 8357c4d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
+++ /dev/null
@@ -1,51 +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.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.datastream.temporal.{ TemporalOperator => JTempOp }
-import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
-import org.apache.flink.streaming.api.datastream.temporal.TemporalWindow
-import org.apache.flink.streaming.api.windowing.helper.Timestamp
-
-abstract class TemporalOperator[I1, I2, OP <: TemporalWindow[OP]](
-  i1: JavaStream[I1], i2: JavaStream[I2]) extends JTempOp[I1, I2, OP](i1, i2) {
-
-  def onWindow(length: Long, ts1: I1 => Long, ts2: I2 => Long, startTime: Long = 0): OP = {
-    val timeStamp1 = getTS(ts1)
-    val timeStamp2 = getTS(ts2)
-    onWindow(length, timeStamp1, timeStamp2, startTime)
-  }
-
-  def getTS[R](ts: R => Long): Timestamp[R] = {
-    val cleanFun = clean(ts)
-    new Timestamp[R] {
-      def getTimestamp(in: R) = cleanFun(in)
-    }
-  }
-
-  /**
-   * 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(i1.getExecutionEnvironment).scalaClean(f)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
new file mode 100644
index 0000000..7232309
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
@@ -0,0 +1,274 @@
+/*
+ * 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.streaming.api.functions.TimestampExtractor
+import org.apache.flink.streaming.api.functions.sink.SinkFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.junit.Test
+import org.junit.Assert._
+
+import scala.collection.mutable
+
+class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testCoGroup(): Unit = {
+    CoGroupJoinITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(1)
+    env.getConfig.enableTimestamps
+    
+    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+        ctx.collect(("a", 0))
+        ctx.collect(("a", 1))
+        ctx.collect(("a", 2))
+        ctx.collect(("b", 3))
+        ctx.collect(("b", 4))
+        ctx.collect(("b", 5))
+        ctx.collect(("a", 6))
+        ctx.collect(("a", 7))
+        ctx.collect(("a", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new CoGroupJoinITCase.Tuple2TimestampExtractor)
+
+    val source2 = env.addSource(new SourceFunction[(String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+        ctx.collect(("a", 0))
+        ctx.collect(("a", 1))
+        ctx.collect(("b", 3))
+        ctx.collect(("c", 6))
+        ctx.collect(("c", 7))
+        ctx.collect(("c", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new CoGroupJoinITCase.Tuple2TimestampExtractor)
+
+    source1.coGroup(source2)
+      .where(_._1)
+      .equalTo(_._1)
+      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+      .apply { (first: Iterator[(String, Int)], second: Iterator[(String, Int)]) =>
+          "F:" + first.mkString("") + " S:" + second.mkString("")
+      }
+      .addSink(new SinkFunction[String]() {
+        def invoke(value: String) {
+          CoGroupJoinITCase.testResults += value
+        }
+      })
+
+    env.execute("CoGroup Test")
+
+    val expectedResult = mutable.MutableList(
+      "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+      "F:(b,3)(b,4)(b,5) S:(b,3)",
+      "F:(a,6)(a,7)(a,8) S:",
+      "F: S:(c,6)(c,7)(c,8)")
+
+    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+  }
+
+  @Test
+  def testJoin(): Unit = {
+    CoGroupJoinITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(1)
+    env.getConfig.enableTimestamps
+
+    val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+        ctx.collect(("a", "x", 0))
+        ctx.collect(("a", "y", 1))
+        ctx.collect(("a", "z", 2))
+
+        ctx.collect(("b", "u", 3))
+        ctx.collect(("b", "w", 5))
+
+        ctx.collect(("a", "i", 6))
+        ctx.collect(("a", "j", 7))
+        ctx.collect(("a", "k", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+    val source2 = env.addSource(new SourceFunction[(String, String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+        ctx.collect(("a", "u", 0))
+        ctx.collect(("a", "w", 1))
+
+        ctx.collect(("b", "i", 3))
+        ctx.collect(("b", "k", 5))
+
+        ctx.collect(("a", "x", 6))
+        ctx.collect(("a", "z", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+    source1.join(source2)
+      .where(_._1)
+      .equalTo(_._1)
+      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+      .apply( (l, r) => l.toString + ":" + r.toString)
+      .addSink(new SinkFunction[String]() {
+        def invoke(value: String) {
+          CoGroupJoinITCase.testResults += value
+        }
+      })
+
+    env.execute("Join Test")
+
+    val expectedResult = mutable.MutableList(
+      "(a,x,0):(a,u,0)",
+      "(a,x,0):(a,w,1)",
+      "(a,y,1):(a,u,0)",
+      "(a,y,1):(a,w,1)",
+      "(a,z,2):(a,u,0)",
+      "(a,z,2):(a,w,1)",
+      "(b,u,3):(b,i,3)",
+      "(b,u,3):(b,k,5)",
+      "(b,w,5):(b,i,3)",
+      "(b,w,5):(b,k,5)",
+      "(a,i,6):(a,x,6)",
+      "(a,i,6):(a,z,8)",
+      "(a,j,7):(a,x,6)",
+      "(a,j,7):(a,z,8)",
+      "(a,k,8):(a,x,6)",
+      "(a,k,8):(a,z,8)")
+
+    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+  }
+
+  @Test
+  def testSelfJoin(): Unit = {
+    CoGroupJoinITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(1)
+    env.getConfig.enableTimestamps
+
+    val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
+      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+        ctx.collect(("a", "x", 0))
+        ctx.collect(("a", "y", 1))
+        ctx.collect(("a", "z", 2))
+
+        ctx.collect(("b", "u", 3))
+        ctx.collect(("b", "w", 5))
+
+        ctx.collect(("a", "i", 6))
+        ctx.collect(("a", "j", 7))
+        ctx.collect(("a", "k", 8))
+      }
+
+      def cancel() {
+      }
+    }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+    source1.join(source1)
+      .where(_._1)
+      .equalTo(_._1)
+      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+      .apply( (l, r) => l.toString + ":" + r.toString)
+      .addSink(new SinkFunction[String]() {
+      def invoke(value: String) {
+        CoGroupJoinITCase.testResults += value
+      }
+    })
+
+    env.execute("Self-Join Test")
+
+    val expectedResult = mutable.MutableList(
+      "(a,x,0):(a,x,0)",
+      "(a,x,0):(a,y,1)",
+      "(a,x,0):(a,z,2)",
+      "(a,y,1):(a,x,0)",
+      "(a,y,1):(a,y,1)",
+      "(a,y,1):(a,z,2)",
+      "(a,z,2):(a,x,0)",
+      "(a,z,2):(a,y,1)",
+      "(a,z,2):(a,z,2)",
+      "(b,u,3):(b,u,3)",
+      "(b,u,3):(b,w,5)",
+      "(b,w,5):(b,u,3)",
+      "(b,w,5):(b,w,5)",
+      "(a,i,6):(a,i,6)",
+      "(a,i,6):(a,j,7)",
+      "(a,i,6):(a,k,8)",
+      "(a,j,7):(a,i,6)",
+      "(a,j,7):(a,j,7)",
+      "(a,j,7):(a,k,8)",
+      "(a,k,8):(a,i,6)",
+      "(a,k,8):(a,j,7)",
+      "(a,k,8):(a,k,8)")
+
+    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+  }
+
+}
+
+
+object CoGroupJoinITCase {
+  private var testResults: mutable.MutableList[String] = null
+
+  private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
+    def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
+      element._2
+    }
+
+    def emitWatermark(element: (String, Int), currentTimestamp: Long): Long = {
+      element._2 - 1
+    }
+
+    def getCurrentWatermark: Long = {
+      Long.MinValue
+    }
+  }
+
+  private class Tuple3TimestampExtractor extends TimestampExtractor[(String, String, Int)] {
+    def extractTimestamp(element: (String, String, Int), currentTimestamp: Long): Long = {
+      element._3
+    }
+
+    def emitWatermark(element: (String, String, Int), currentTimestamp: Long): Long = {
+      element._3 - 1
+    }
+
+    def getCurrentWatermark: Long = {
+      Long.MinValue
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/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 d1fd233..c6bd87a 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
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.scala
 import java.lang.reflect.Method
 
 import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
+import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, JoinedStreams}
 
 import scala.language.existentials
 
@@ -138,14 +138,14 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       classOf[KeyedStream[_, _]])
 
     checkMethods(
-      "StreamJoinOperator", "StreamJoinOperator",
-      classOf[org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator[_,_]],
-      classOf[StreamJoinOperator[_,_]])
+      "JoinedStreams.WithWindow", "JoinedStreams.WithWindow",
+      classOf[org.apache.flink.streaming.api.datastream.JoinedStreams.WithWindow[_,_,_,_]],
+      classOf[JoinedStreams.WithWindow[_,_,_,_]])
 
     checkMethods(
-      "TemporalOperator", "TemporalOperator",
-      classOf[org.apache.flink.streaming.api.datastream.temporal.TemporalOperator[_,_,_]],
-      classOf[TemporalOperator[_,_,_]])
+      "CoGroupedStreams.WithWindow", "CoGroupedStreams.WithWindow",
+      classOf[org.apache.flink.streaming.api.datastream.CoGroupedStreams.WithWindow[_,_,_,_]],
+      classOf[CoGroupedStreams.WithWindow[_,_,_,_]])
 
     checkMethods(
       "WindowedDataStream", "WindowedDataStream",


[7/8] flink git commit: Move CoGroupJoinITCase to windowing package

Posted by al...@apache.org.
Move CoGroupJoinITCase to windowing package


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

Branch: refs/heads/master
Commit: ce792b11a4a2b8d7b02a59dcadcc4c052ff5531a
Parents: ff367d6
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Oct 7 16:18:24 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/CoGroupJoinITCase.java  | 372 ------------------
 .../operators/windowing/CoGroupJoinITCase.java  | 373 +++++++++++++++++++
 2 files changed, 373 insertions(+), 372 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ce792b11/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
deleted file mode 100644
index 9ddd6eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
+++ /dev/null
@@ -1,372 +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;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
-
-	private static List<String> testResults;
-
-	@Test
-	public void testCoGroup() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-				ctx.collect(Tuple2.of("a", 2));
-
-				ctx.collect(Tuple2.of("b", 3));
-				ctx.collect(Tuple2.of("b", 4));
-				ctx.collect(Tuple2.of("b", 5));
-
-				ctx.collect(Tuple2.of("a", 6));
-				ctx.collect(Tuple2.of("a", 7));
-				ctx.collect(Tuple2.of("a", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).extractTimestamp(new Tuple2TimestampExtractor());
-
-		DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-
-				ctx.collect(Tuple2.of("b", 3));
-
-				ctx.collect(Tuple2.of("c", 6));
-				ctx.collect(Tuple2.of("c", 7));
-				ctx.collect(Tuple2.of("c", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).extractTimestamp(new Tuple2TimestampExtractor());
-
-
-		source1.coGroup(source2)
-				.where(new Tuple2KeyExtractor())
-				.equalTo(new Tuple2KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
-					@Override
-					public void coGroup(Iterable<Tuple2<String, Integer>> first,
-							Iterable<Tuple2<String, Integer>> second,
-							Collector<String> out) throws Exception {
-						StringBuilder result = new StringBuilder();
-						result.append("F:");
-						for (Tuple2<String, Integer> t: first) {
-							result.append(t.toString());
-						}
-						result.append(" S:");
-						for (Tuple2<String, Integer> t: second) {
-							result.append(t.toString());
-						}
-						out.collect(result.toString());
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("CoGroup Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
-				"F:(b,3)(b,4)(b,5) S:(b,3)",
-				"F:(a,6)(a,7)(a,8) S:",
-				"F: S:(c,6)(c,7)(c,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	@Test
-	public void testJoin() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "x", 0));
-				ctx.collect(Tuple3.of("a", "y", 1));
-				ctx.collect(Tuple3.of("a", "z", 2));
-
-				ctx.collect(Tuple3.of("b", "u", 3));
-				ctx.collect(Tuple3.of("b", "w", 5));
-
-				ctx.collect(Tuple3.of("a", "i", 6));
-				ctx.collect(Tuple3.of("a", "j", 7));
-				ctx.collect(Tuple3.of("a", "k", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).extractTimestamp(new Tuple3TimestampExtractor());
-
-		DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "u", 0));
-				ctx.collect(Tuple3.of("a", "w", 1));
-
-				ctx.collect(Tuple3.of("b", "i", 3));
-				ctx.collect(Tuple3.of("b", "k", 5));
-
-				ctx.collect(Tuple3.of("a", "x", 6));
-				ctx.collect(Tuple3.of("a", "z", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).extractTimestamp(new Tuple3TimestampExtractor());
-
-
-		source1.join(source2)
-				.where(new Tuple3KeyExtractor())
-				.equalTo(new Tuple3KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
-					@Override
-					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
-						return first + ":" + second;
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("Join Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(a,x,0):(a,u,0)",
-				"(a,x,0):(a,w,1)",
-				"(a,y,1):(a,u,0)",
-				"(a,y,1):(a,w,1)",
-				"(a,z,2):(a,u,0)",
-				"(a,z,2):(a,w,1)",
-				"(b,u,3):(b,i,3)",
-				"(b,u,3):(b,k,5)",
-				"(b,w,5):(b,i,3)",
-				"(b,w,5):(b,k,5)",
-				"(a,i,6):(a,x,6)",
-				"(a,i,6):(a,z,8)",
-				"(a,j,7):(a,x,6)",
-				"(a,j,7):(a,z,8)",
-				"(a,k,8):(a,x,6)",
-				"(a,k,8):(a,z,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	@Test
-	public void testSelfJoin() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "x", 0));
-				ctx.collect(Tuple3.of("a", "y", 1));
-				ctx.collect(Tuple3.of("a", "z", 2));
-
-				ctx.collect(Tuple3.of("b", "u", 3));
-				ctx.collect(Tuple3.of("b", "w", 5));
-
-				ctx.collect(Tuple3.of("a", "i", 6));
-				ctx.collect(Tuple3.of("a", "j", 7));
-				ctx.collect(Tuple3.of("a", "k", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).extractTimestamp(new Tuple3TimestampExtractor());
-
-		source1.join(source1)
-				.where(new Tuple3KeyExtractor())
-				.equalTo(new Tuple3KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
-					@Override
-					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
-						return first + ":" + second;
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("Self-Join Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(a,x,0):(a,x,0)",
-				"(a,x,0):(a,y,1)",
-				"(a,x,0):(a,z,2)",
-				"(a,y,1):(a,x,0)",
-				"(a,y,1):(a,y,1)",
-				"(a,y,1):(a,z,2)",
-				"(a,z,2):(a,x,0)",
-				"(a,z,2):(a,y,1)",
-				"(a,z,2):(a,z,2)",
-				"(b,u,3):(b,u,3)",
-				"(b,u,3):(b,w,5)",
-				"(b,w,5):(b,u,3)",
-				"(b,w,5):(b,w,5)",
-				"(a,i,6):(a,i,6)",
-				"(a,i,6):(a,j,7)",
-				"(a,i,6):(a,k,8)",
-				"(a,j,7):(a,i,6)",
-				"(a,j,7):(a,j,7)",
-				"(a,j,7):(a,k,8)",
-				"(a,k,8):(a,i,6)",
-				"(a,k,8):(a,j,7)",
-				"(a,k,8):(a,k,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1;
-		}
-
-		@Override
-		public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
-			return element.f2;
-		}
-
-		@Override
-		public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
-			return element.f2 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class Tuple2KeyExtractor 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;
-		}
-	}
-
-	private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple3<String, String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ce792b11/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
new file mode 100644
index 0000000..bb79e5e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
@@ -0,0 +1,373 @@
+/*
+* 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 com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+	private static List<String> testResults;
+
+	@Test
+	public void testCoGroup() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+				ctx.collect(Tuple2.of("a", 2));
+
+				ctx.collect(Tuple2.of("b", 3));
+				ctx.collect(Tuple2.of("b", 4));
+				ctx.collect(Tuple2.of("b", 5));
+
+				ctx.collect(Tuple2.of("a", 6));
+				ctx.collect(Tuple2.of("a", 7));
+				ctx.collect(Tuple2.of("a", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+		DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple2.of("a", 0));
+				ctx.collect(Tuple2.of("a", 1));
+
+				ctx.collect(Tuple2.of("b", 3));
+
+				ctx.collect(Tuple2.of("c", 6));
+				ctx.collect(Tuple2.of("c", 7));
+				ctx.collect(Tuple2.of("c", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple2TimestampExtractor());
+
+
+		source1.coGroup(source2)
+				.where(new Tuple2KeyExtractor())
+				.equalTo(new Tuple2KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+					@Override
+					public void coGroup(Iterable<Tuple2<String, Integer>> first,
+							Iterable<Tuple2<String, Integer>> second,
+							Collector<String> out) throws Exception {
+						StringBuilder result = new StringBuilder();
+						result.append("F:");
+						for (Tuple2<String, Integer> t: first) {
+							result.append(t.toString());
+						}
+						result.append(" S:");
+						for (Tuple2<String, Integer> t: second) {
+							result.append(t.toString());
+						}
+						out.collect(result.toString());
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("CoGroup Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+				"F:(b,3)(b,4)(b,5) S:(b,3)",
+				"F:(a,6)(a,7)(a,8) S:",
+				"F: S:(c,6)(c,7)(c,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	@Test
+	public void testJoin() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "x", 0));
+				ctx.collect(Tuple3.of("a", "y", 1));
+				ctx.collect(Tuple3.of("a", "z", 2));
+
+				ctx.collect(Tuple3.of("b", "u", 3));
+				ctx.collect(Tuple3.of("b", "w", 5));
+
+				ctx.collect(Tuple3.of("a", "i", 6));
+				ctx.collect(Tuple3.of("a", "j", 7));
+				ctx.collect(Tuple3.of("a", "k", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+		DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "u", 0));
+				ctx.collect(Tuple3.of("a", "w", 1));
+
+				ctx.collect(Tuple3.of("b", "i", 3));
+				ctx.collect(Tuple3.of("b", "k", 5));
+
+				ctx.collect(Tuple3.of("a", "x", 6));
+				ctx.collect(Tuple3.of("a", "z", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+
+		source1.join(source2)
+				.where(new Tuple3KeyExtractor())
+				.equalTo(new Tuple3KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+					@Override
+					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+						return first + ":" + second;
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("Join Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(a,x,0):(a,u,0)",
+				"(a,x,0):(a,w,1)",
+				"(a,y,1):(a,u,0)",
+				"(a,y,1):(a,w,1)",
+				"(a,z,2):(a,u,0)",
+				"(a,z,2):(a,w,1)",
+				"(b,u,3):(b,i,3)",
+				"(b,u,3):(b,k,5)",
+				"(b,w,5):(b,i,3)",
+				"(b,w,5):(b,k,5)",
+				"(a,i,6):(a,x,6)",
+				"(a,i,6):(a,z,8)",
+				"(a,j,7):(a,x,6)",
+				"(a,j,7):(a,z,8)",
+				"(a,k,8):(a,x,6)",
+				"(a,k,8):(a,z,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	@Test
+	public void testSelfJoin() throws Exception {
+
+		testResults = Lists.newArrayList();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+				ctx.collect(Tuple3.of("a", "x", 0));
+				ctx.collect(Tuple3.of("a", "y", 1));
+				ctx.collect(Tuple3.of("a", "z", 2));
+
+				ctx.collect(Tuple3.of("b", "u", 3));
+				ctx.collect(Tuple3.of("b", "w", 5));
+
+				ctx.collect(Tuple3.of("a", "i", 6));
+				ctx.collect(Tuple3.of("a", "j", 7));
+				ctx.collect(Tuple3.of("a", "k", 8));
+			}
+
+			@Override
+			public void cancel() {
+			}
+		}).extractTimestamp(new Tuple3TimestampExtractor());
+
+		source1.join(source1)
+				.where(new Tuple3KeyExtractor())
+				.equalTo(new Tuple3KeyExtractor())
+				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+					@Override
+					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+						return first + ":" + second;
+					}
+				})
+				.addSink(new SinkFunction<String>() {
+					@Override
+					public void invoke(String value) throws Exception {
+						testResults.add(value);
+					}
+				});
+
+		env.execute("Self-Join Test");
+
+		List<String> expectedResult = Lists.newArrayList(
+				"(a,x,0):(a,x,0)",
+				"(a,x,0):(a,y,1)",
+				"(a,x,0):(a,z,2)",
+				"(a,y,1):(a,x,0)",
+				"(a,y,1):(a,y,1)",
+				"(a,y,1):(a,z,2)",
+				"(a,z,2):(a,x,0)",
+				"(a,z,2):(a,y,1)",
+				"(a,z,2):(a,z,2)",
+				"(b,u,3):(b,u,3)",
+				"(b,u,3):(b,w,5)",
+				"(b,w,5):(b,u,3)",
+				"(b,w,5):(b,w,5)",
+				"(a,i,6):(a,i,6)",
+				"(a,i,6):(a,j,7)",
+				"(a,i,6):(a,k,8)",
+				"(a,j,7):(a,i,6)",
+				"(a,j,7):(a,j,7)",
+				"(a,j,7):(a,k,8)",
+				"(a,k,8):(a,i,6)",
+				"(a,k,8):(a,j,7)",
+				"(a,k,8):(a,k,8)");
+
+		Collections.sort(expectedResult);
+		Collections.sort(testResults);
+
+		Assert.assertEquals(expectedResult, testResults);
+	}
+
+	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1;
+		}
+
+		@Override
+		public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+			return element.f1 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
+			return element.f2;
+		}
+
+		@Override
+		public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
+			return element.f2 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class Tuple2KeyExtractor 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;
+		}
+	}
+
+	private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple3<String, String, Integer> value) throws Exception {
+			return value.f0;
+		}
+	}
+
+}


[3/8] flink git commit: [FLINK-2550] Rework interplay of Window Assigners and TimeCharacteristic

Posted by al...@apache.org.
[FLINK-2550] Rework interplay of Window Assigners and TimeCharacteristic


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

Branch: refs/heads/master
Commit: ff367d6ea728a7c5bc334f34591a4d79e573972f
Parents: 28a38bb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 18:19:56 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       |  17 +--
 .../streaming/api/datastream/DataStream.java    |  20 +---
 .../streaming/api/datastream/KeyedStream.java   |  20 +---
 .../api/datastream/WindowedStream.java          |  29 ++---
 .../environment/StreamExecutionEnvironment.java |  12 +++
 .../api/windowing/assigners/GlobalWindows.java  |   3 +-
 .../assigners/SlidingProcessingTimeWindows.java | 106 -------------------
 .../windowing/assigners/SlidingTimeWindows.java |  11 +-
 .../TumblingProcessingTimeWindows.java          |  81 --------------
 .../assigners/TumblingTimeWindows.java          |  11 +-
 .../api/windowing/assigners/WindowAssigner.java |   3 +-
 .../operators/windowing/WindowOperator.java     |   5 +
 .../flink/streaming/api/CoGroupJoinITCase.java  |   6 +-
 .../windowing/AllWindowTranslationTest.java     |  60 +++++++----
 .../windowing/WindowTranslationTest.java        |  89 +++++++++++++---
 .../streaming/examples/join/WindowJoin.java     |   3 +-
 .../scala/examples/join/WindowJoin.scala        |   3 +-
 .../flink/streaming/api/scala/DataStream.scala  |  37 +------
 .../flink/streaming/api/scala/KeyedStream.scala |  37 +------
 .../api/scala/StreamExecutionEnvironment.scala  |   5 +
 .../api/scala/AllWindowTranslationTest.scala    |  24 +++--
 .../streaming/api/scala/CoGroupJoinITCase.scala |   9 +-
 .../api/scala/WindowTranslationTest.scala       |  22 ++--
 23 files changed, 233 insertions(+), 380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 89c4857..a8d7654 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.TimeCharacteristic;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
@@ -80,7 +81,7 @@ public class AllWindowedStream<T, W extends Window> {
 			WindowAssigner<? super T, W> windowAssigner) {
 		this.input = input;
 		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger();
+		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
 	}
 
 	/**
@@ -139,12 +140,14 @@ public class AllWindowedStream<T, W extends Window> {
 
 		OneInputStreamOperator<T, T> operator;
 
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
 		if (evictor != null) {
 			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
 					new HeapWindowBuffer.Factory<T>(),
 					new ReduceAllWindowFunction<W, T>(function),
 					trigger,
-					evictor);
+					evictor).enableSetProcessingTime(setProcessingTime);
 
 		} else {
 			// we need to copy because we need our own instance of the pre aggregator
@@ -154,7 +157,7 @@ public class AllWindowedStream<T, W extends Window> {
 			operator = new NonKeyedWindowOperator<>(windowAssigner,
 					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
 					new ReduceAllWindowFunction<W, T>(function),
-					trigger);
+					trigger).enableSetProcessingTime(setProcessingTime);
 		}
 
 		return input.transform(opName, input.getType(), operator).setParallelism(1);
@@ -205,20 +208,22 @@ public class AllWindowedStream<T, W extends Window> {
 
 		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
 
-		OneInputStreamOperator<T, R> operator;
+		NonKeyedWindowOperator<T, R, W> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
 
 		if (evictor != null) {
 			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
 					new HeapWindowBuffer.Factory<T>(),
 					function,
 					trigger,
-					evictor);
+					evictor).enableSetProcessingTime(setProcessingTime);
 
 		} else {
 			operator = new NonKeyedWindowOperator<>(windowAssigner,
 					new HeapWindowBuffer.Factory<T>(),
 					function,
-					trigger);
+					trigger).enableSetProcessingTime(setProcessingTime);
 		}
 
 		return input.transform(opName, resultType, operator).setParallelism(1);

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 0be1d56..ee8b3d2 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
@@ -59,9 +59,7 @@ import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.PartitionTransformation;
 import org.apache.flink.streaming.api.transformations.StreamTransformation;
 import org.apache.flink.streaming.api.transformations.UnionTransformation;
-import org.apache.flink.streaming.api.windowing.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.helper.Count;
@@ -72,7 +70,6 @@ import org.apache.flink.streaming.api.windowing.helper.WindowingHelper;
 import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
 import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
 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.operators.ExtractTimestampsOperator;
@@ -726,13 +723,7 @@ public class DataStream<T> {
 	 * @param size The size of the window.
 	 */
 	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
-		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
-		if (actualSize instanceof EventTime) {
-			return windowAll(TumblingTimeWindows.of(actualSize));
-		} else {
-			return windowAll(TumblingProcessingTimeWindows.of(actualSize));
-		}
+		return windowAll(TumblingTimeWindows.of(size));
 	}
 
 	/**
@@ -747,14 +738,7 @@ public class DataStream<T> {
 	 * @param size The size of the window.
 	 */
 	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
-		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
-		if (actualSize instanceof EventTime) {
-			return windowAll(SlidingTimeWindows.of(size, slide));
-		} else {
-			return windowAll(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
-		}
+		return windowAll(SlidingTimeWindows.of(size, slide));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 edb7981..2e6d7d6 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
@@ -32,13 +32,10 @@ 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;
 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;
@@ -122,13 +119,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 	 * @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));
-		} else {
-			return window(TumblingProcessingTimeWindows.of(actualSize));
-		}
+		return window(TumblingTimeWindows.of(size));
 	}
 
 	/**
@@ -143,14 +134,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
 	 * @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, actualSlide));
-		} else {
-			return window(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
-		}
+		return window(SlidingTimeWindows.of(size, slide));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 1273b42..99f7d06 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.TimeCharacteristic;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
@@ -32,8 +33,8 @@ import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
 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.SlidingTimeWindows;
+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.evictors.Evictor;
 import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
@@ -91,7 +92,7 @@ public class WindowedStream<T, K, W extends Window> {
 			WindowAssigner<? super T, W> windowAssigner) {
 		this.input = input;
 		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger();
+		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
 	}
 
 	/**
@@ -151,13 +152,15 @@ public class WindowedStream<T, K, W extends Window> {
 
 		OneInputStreamOperator<T, T> operator;
 
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
 		if (evictor != null) {
 			operator = new EvictingWindowOperator<>(windowAssigner,
 					keySel,
 					new HeapWindowBuffer.Factory<T>(),
 					new ReduceWindowFunction<K, W, T>(function),
 					trigger,
-					evictor);
+					evictor).enableSetProcessingTime(setProcessingTime);
 
 		} else {
 			// we need to copy because we need our own instance of the pre aggregator
@@ -168,7 +171,7 @@ public class WindowedStream<T, K, W extends Window> {
 					keySel,
 					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
 					new ReduceWindowFunction<K, W, T>(function),
-					trigger);
+					trigger).enableSetProcessingTime(setProcessingTime);
 		}
 
 		return input.transform(opName, input.getType(), operator);
@@ -222,7 +225,9 @@ public class WindowedStream<T, K, W extends Window> {
 		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
 		KeySelector<T, K> keySel = input.getKeySelector();
 
-		OneInputStreamOperator<T, R> operator;
+		WindowOperator<K, T, R, W> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
 
 		if (evictor != null) {
 			operator = new EvictingWindowOperator<>(windowAssigner,
@@ -230,14 +235,14 @@ public class WindowedStream<T, K, W extends Window> {
 					new HeapWindowBuffer.Factory<T>(),
 					function,
 					trigger,
-					evictor);
+					evictor).enableSetProcessingTime(setProcessingTime);
 
 		} else {
 			operator = new WindowOperator<>(windowAssigner,
 					keySel,
 					new HeapWindowBuffer.Factory<T>(),
 					function,
-					trigger);
+					trigger).enableSetProcessingTime(setProcessingTime);;
 		}
 
 		return input.transform(opName, resultType, operator);
@@ -450,8 +455,8 @@ public class WindowedStream<T, K, W extends Window> {
 			TypeInformation<R> resultType,
 			String functionName) {
 
-		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
+		if (windowAssigner instanceof SlidingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			SlidingTimeWindows timeWindows = (SlidingTimeWindows) windowAssigner;
 			final long windowLength = timeWindows.getSize();
 			final long windowSlide = timeWindows.getSlide();
 
@@ -475,8 +480,8 @@ public class WindowedStream<T, K, W extends Window> {
 						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;
+		} else if (windowAssigner instanceof TumblingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			TumblingTimeWindows timeWindows = (TumblingTimeWindows) windowAssigner;
 			final long windowLength = timeWindows.getSize();
 			final long windowSlide = timeWindows.getSize();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index c2e2880..cc96217 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -541,11 +541,23 @@ public abstract class StreamExecutionEnvironment {
 	/**
 	 * Sets the time characteristic for all streams create from this environment, e.g., processing
 	 * time, event time, or ingestion time.
+	 *
+	 * <p>
+	 * If you set the characteristic to IngestionTime of EventTime this will set a default
+	 * watermark update interval of 200 ms. If this is not applicable for your application
+	 * you should change it using {@link ExecutionConfig#setAutoWatermarkInterval(long)}.
 	 * 
 	 * @param characteristic The time characteristic.
 	 */
 	public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) {
 		this.timeCharacteristic = Objects.requireNonNull(characteristic);
+		if (characteristic == TimeCharacteristic.ProcessingTime) {
+			getConfig().disableTimestamps();
+			getConfig().setAutoWatermarkInterval(0);
+		} else {
+			getConfig().enableTimestamps();
+			getConfig().setAutoWatermarkInterval(200);
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 52c8f55..dbeb5ce 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
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 
@@ -42,7 +43,7 @@ public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
 	}
 
 	@Override
-	public Trigger<Object, GlobalWindow> getDefaultTrigger() {
+	public Trigger<Object, GlobalWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
 		return null;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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
deleted file mode 100644
index 65d7641..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
+++ /dev/null
@@ -1,106 +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.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;
-
-import java.io.IOException;
-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;
-
-	private final long size;
-
-	private final long slide;
-
-	private transient List<TimeWindow> result;
-
-	private SlidingProcessingTimeWindows(long size, long slide) {
-		this.size = size;
-		this.slide = slide;
-		this.result = Lists.newArrayListWithCapacity((int) (size / slide));
-	}
-
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		this.result = Lists.newArrayListWithCapacity((int) (size / slide));
-	}
-
-	@Override
-	public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
-		result.clear();
-		long time = System.currentTimeMillis();
-		long lastStart = time - time % slide;
-		for (long start = lastStart;
-			start > time - size;
-			start -= slide) {
-			result.add(new TimeWindow(start, size));
-		}
-		return result;
-	}
-
-	public long getSize() {
-		return size;
-	}
-
-	public long getSlide() {
-		return slide;
-	}
-
-	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger() {
-		return ProcessingTimeTrigger.create();
-	}
-
-	@Override
-	public String toString() {
-		return "SlidingProcessingTimeWindows(" + size + ", " + slide + ")";
-	}
-
-	/**
-	 * Creates a new {@code SlidingProcessingTimeWindows} {@link WindowAssigner} that assigns
-	 * elements to sliding time windows based on the current processing time.
-	 *
-	 * @param size The size of the generated windows.
-	 * @param slide The slide interval of the generated windows.
-	 * @return The time policy.
-	 */
-	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/ff367d6e/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 52ae356..6036dfb 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,7 +17,10 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 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.triggers.WatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -72,8 +75,12 @@ public class SlidingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	}
 
 	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger() {
-		return WatermarkTrigger.create();
+	public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
+		if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
+			return ProcessingTimeTrigger.create();
+		} else {
+			return WatermarkTrigger.create();
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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
deleted file mode 100644
index 41f6362..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
+++ /dev/null
@@ -1,81 +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.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;
-
-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;
-
-	private long size;
-
-	private TumblingProcessingTimeWindows(long size) {
-		this.size = size;
-	}
-
-	@Override
-	public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
-		long time = System.currentTimeMillis();
-		long start = time - (time % size);
-		return Collections.singletonList(new TimeWindow(start, size));
-	}
-
-	public long getSize() {
-		return size;
-	}
-
-	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger() {
-		return ProcessingTimeTrigger.create();
-	}
-
-	@Override
-	public String toString() {
-		return "TumblingProcessingTimeWindows(" + size + ")";
-	}
-
-	/**
-	 * Creates a new {@code TumblingProcessingTimeWindows} {@link WindowAssigner} that assigns
-	 * elements to time windows based on the current processing time.
-	 *
-	 * @param size The size of the generated windows.
-	 * @return The time policy.
-	 */
-	public static TumblingProcessingTimeWindows of(AbstractTime size) {
-		return new TumblingProcessingTimeWindows(size.toMilliseconds());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 b6022b3..d57dc33 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,7 +17,10 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 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.triggers.WatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -58,8 +61,12 @@ public class TumblingTimeWindows extends WindowAssigner<Object, TimeWindow> {
 	}
 
 	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger() {
-		return WatermarkTrigger.create();
+	public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
+		if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
+			return ProcessingTimeTrigger.create();
+		} else {
+			return WatermarkTrigger.create();
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 20fe365..d0b1ed0 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
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.assigners;
 
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import java.io.Serializable;
@@ -50,5 +51,5 @@ public abstract class WindowAssigner<T, W extends Window> implements Serializabl
 	/**
 	 * Returns the default trigger associated with this {@code WindowAssigner}.
 	 */
-	public abstract Trigger<T, W> getDefaultTrigger();
+	public abstract Trigger<T, W> getDefaultTrigger(StreamExecutionEnvironment env);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 548afb3..368b8fa 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
@@ -347,6 +347,11 @@ public class WindowOperator<K, IN, OUT, W extends Window>
 	// ------------------------------------------------------------------------
 
 	@VisibleForTesting
+	public boolean isSetProcessingTime() {
+		return setProcessingTime;
+	}
+
+	@VisibleForTesting
 	public Trigger<? super IN, ? super W> getTriggerTemplate() {
 		return triggerTemplate;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
index c06a608..9ddd6eb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
@@ -49,8 +49,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 		testResults = Lists.newArrayList();
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(1);
-		env.getConfig().enableTimestamps();
 
 		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
 			private static final long serialVersionUID = 1L;
@@ -144,8 +144,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 		testResults = Lists.newArrayList();
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(1);
-		env.getConfig().enableTimestamps();
 
 		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
 			private static final long serialVersionUID = 1L;
@@ -239,8 +239,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 		testResults = Lists.newArrayList();
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(1);
-		env.getConfig().enableTimestamps();
 
 		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
 			private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 09a7149..4fa16ac 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
@@ -19,25 +19,25 @@ 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.TimeCharacteristic;
 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.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
 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.triggers.WatermarkTrigger;
 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;
 
 import java.util.concurrent.TimeUnit;
@@ -50,30 +50,33 @@ import java.util.concurrent.TimeUnit;
 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
+	 * These tests ensure that the correct trigger is set when using event-time windows.
 	 */
-	@Ignore
 	@Test
-	public void testFastTimeWindows() throws Exception {
+	@SuppressWarnings("rawtypes")
+	public void testEventTime() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
 
 		DummyReducer reducer = new DummyReducer();
 
 		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.reduce(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);
+		Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+		Assert.assertFalse(winOperator1.isSetProcessingTime());
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
 
@@ -88,20 +91,26 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 
 		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);
+		Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+		Assert.assertFalse(winOperator2.isSetProcessingTime());
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 	}
 
 	@Test
 	@SuppressWarnings("rawtypes")
 	public void testNonEvicting() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
 		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(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
 				.reduce(reducer);
 
@@ -109,12 +118,13 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
 		Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
 		NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.isSetProcessingTime());
 		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+				.windowAll(TumblingTimeWindows.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;
@@ -132,8 +142,9 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
 		Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
 		NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+		Assert.assertTrue(winOperator1.isSetProcessingTime());
 		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
 		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 	}
 
@@ -141,13 +152,14 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 	@SuppressWarnings("rawtypes")
 	public void testEvicting() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
 
 		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(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
 				.reduce(reducer);
 
@@ -155,13 +167,14 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		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.assertFalse(winOperator1.isSetProcessingTime());
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
 		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+				.windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
 				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
@@ -180,8 +193,9 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
 		Assert.assertTrue(operator2 instanceof EvictingNonKeyedWindowOperator);
 		EvictingNonKeyedWindowOperator winOperator2 = (EvictingNonKeyedWindowOperator) operator2;
+		Assert.assertFalse(winOperator2.isSetProcessingTime());
 		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
 		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
 		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 5124add..10fe734 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
@@ -20,19 +20,20 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 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.TimeCharacteristic;
 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.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.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
 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.triggers.WatermarkTrigger;
 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;
@@ -59,13 +60,13 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
 		DummyReducer reducer = new DummyReducer();
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS),
-						Time.of(100, TimeUnit.MILLISECONDS)))
+				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.reduce(reducer);
 
 		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
@@ -74,7 +75,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.window(SlidingTimeWindows.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;
 
@@ -92,10 +93,63 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
 	}
 
+	/**
+	 * These tests ensure that the correct trigger is set when using event-time windows.
+	 */
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testEventTime() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.keyBy(0)
+				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.reduce(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 WindowOperator);
+		WindowOperator winOperator1 = (WindowOperator) operator1;
+		Assert.assertFalse(winOperator1.isSetProcessingTime());
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.keyBy(0)
+				.window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void apply(Tuple tuple,
+							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 WindowOperator);
+		WindowOperator winOperator2 = (WindowOperator) operator2;
+		Assert.assertFalse(winOperator2.isSetProcessingTime());
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
 	@Test
 	@SuppressWarnings("rawtypes")
 	public void testNonEvicting() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
 		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
 
@@ -103,7 +157,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.trigger(CountTrigger.of(100))
 				.reduce(reducer);
 
@@ -111,13 +165,14 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
 		Assert.assertTrue(operator1 instanceof WindowOperator);
 		WindowOperator winOperator1 = (WindowOperator) operator1;
+		Assert.assertTrue(winOperator1.isSetProcessingTime());
 		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+				.window(TumblingTimeWindows.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;
@@ -135,8 +190,9 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
 		Assert.assertTrue(operator2 instanceof WindowOperator);
 		WindowOperator winOperator2 = (WindowOperator) operator2;
+		Assert.assertTrue(winOperator2.isSetProcessingTime());
 		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
 		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 	}
 
@@ -144,6 +200,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 	@SuppressWarnings("rawtypes")
 	public void testEvicting() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
 
 		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
 
@@ -151,7 +208,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Tuple2<String, Integer>> window1 = source
 				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
 				.evictor(CountEvictor.of(100))
 				.reduce(reducer);
 
@@ -159,14 +216,15 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
 		Assert.assertTrue(operator1 instanceof EvictingWindowOperator);
 		EvictingWindowOperator winOperator1 = (EvictingWindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertFalse(winOperator1.isSetProcessingTime());
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
 		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
 		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 
 		DataStream<Tuple2<String, Integer>> window2 = source
 				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+				.window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
 				.trigger(CountTrigger.of(100))
 				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
 				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
@@ -185,8 +243,9 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
 		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
 		Assert.assertTrue(operator2 instanceof EvictingWindowOperator);
 		EvictingWindowOperator winOperator2 = (EvictingWindowOperator) operator2;
+		Assert.assertFalse(winOperator2.isSetProcessingTime());
 		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
 		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
 		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
index 8abf9d6..5915a7a 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.TimestampExtractor;
@@ -64,7 +65,7 @@ public class WindowJoin {
 
 		// obtain execution environment
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableTimestamps();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 
 		// connect to the data sources for grades and salaries
 		Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
index 225dab7..42484e8 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.scala.examples.join
 
 import java.util.concurrent.TimeUnit
 
+import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.scala._
 import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
 import org.apache.flink.streaming.api.windowing.time.Time
@@ -45,7 +46,7 @@ object WindowJoin {
     }
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.getConfig.enableTimestamps()
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
 
     //Create streams for grades and salaries by mapping the inputs to the corresponding objects
     val grades = setGradesInput(env)

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 7babc40..fb4d75d 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
@@ -34,7 +34,7 @@ import org.apache.flink.streaming.api.scala.function.StatefulFunction
 import org.apache.flink.streaming.api.windowing.assigners._
 import org.apache.flink.streaming.api.windowing.helper.WindowingHelper
 import org.apache.flink.streaming.api.windowing.policy.{EvictionPolicy, TriggerPolicy}
-import org.apache.flink.streaming.api.windowing.time.{AbstractTime, EventTime, ProcessingTime}
+import org.apache.flink.streaming.api.windowing.time.AbstractTime
 import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window}
 import org.apache.flink.streaming.util.serialization.SerializationSchema
 import org.apache.flink.util.Collector
@@ -624,20 +624,8 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * @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)
-          .asInstanceOf[WindowAssigner[T, TimeWindow]]
-        windowAll(assigner)
-      case t: ProcessingTime =>
-        val assigner = TumblingProcessingTimeWindows.of(actualSize)
-          .asInstanceOf[WindowAssigner[T, TimeWindow]]
-        windowAll(assigner)
-      case _ => throw new RuntimeException("Invalid time: " + actualSize)
-    }
+    val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
+    windowAll(assigner)
   }
 
   /**
@@ -651,23 +639,8 @@ class DataStream[T](javaStream: JavaStream[T]) {
    * @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,
-          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-        windowAll(assigner)
-      case t: ProcessingTime =>
-        val assigner = SlidingProcessingTimeWindows.of(
-          actualSize,
-          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-        windowAll(assigner)
-      case _ => throw new RuntimeException("Invalid time: " + actualSize)
-    }
+    val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
+    windowAll(assigner)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 0ce36aa..c605bb1 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
@@ -24,7 +24,7 @@ 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.time.AbstractTime
 import org.apache.flink.streaming.api.windowing.windows.{Window, TimeWindow}
 import scala.reflect.ClassTag
 import org.apache.flink.api.common.typeinfo.TypeInformation
@@ -50,20 +50,8 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
    * @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)
-          .asInstanceOf[WindowAssigner[T, TimeWindow]]
-        window(assigner)
-      case t: ProcessingTime =>
-        val assigner = TumblingProcessingTimeWindows.of(actualSize)
-          .asInstanceOf[WindowAssigner[T, TimeWindow]]
-        window(assigner)
-      case _ => throw new RuntimeException("Invalid time: " + actualSize)
-    }
+    val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
+    window(assigner)
   }
 
   /**
@@ -78,23 +66,8 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
    * @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,
-          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-        window(assigner)
-      case t: ProcessingTime =>
-        val assigner = SlidingProcessingTimeWindows.of(
-          actualSize,
-          actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-        window(assigner)
-      case _ => throw new RuntimeException("Invalid time: " + actualSize)
-    }
+    val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
+    window(assigner)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 7492e48..f767aba 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -299,6 +299,11 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * Sets the time characteristic for all streams create from this environment, e.g., processing
    * time, event time, or ingestion time.
    *
+   * If you set the characteristic to IngestionTime of EventTime this will set a default
+   * watermark update interval of 200 ms. If this is not applicable for your application
+   * you should change it using
+   * [[org.apache.flink.api.common.ExecutionConfig#setAutoWatermarkInterval(long)]]
+   *
    * @param characteristic The time characteristic.
    */
   def setStreamTimeCharacteristic(characteristic: TimeCharacteristic) : Unit = {

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/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 dece9f6..99fcd07 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
@@ -22,9 +22,10 @@ package org.apache.flink.streaming.api.scala
 import java.util.concurrent.TimeUnit
 
 import org.apache.flink.api.common.functions.RichReduceFunction
+import org.apache.flink.streaming.api.TimeCharacteristic
 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.assigners.{TumblingTimeWindows, SlidingTimeWindows}
 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}
@@ -49,13 +50,14 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
   @Test
   def testFastTimeWindows(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
 
     val source = env.fromElements(("hello", 1), ("hello", 2))
 
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(
+      .windowAll(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .reduce(reducer)
@@ -69,7 +71,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .keyBy(0)
-      .windowAll(SlidingProcessingTimeWindows.of(
+      .windowAll(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
@@ -96,7 +98,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(
+      .windowAll(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .trigger(CountTrigger.of(100))
@@ -110,13 +112,13 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(operator1.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
     val winOperator1 = operator1.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
     assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
     assertTrue(
       winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
 
 
     val window2 = source
-      .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+      .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
       def apply(
@@ -133,7 +135,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(operator2.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
     val winOperator2 = operator2.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
     assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
     assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
   }
 
@@ -146,7 +148,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val reducer = new DummyReducer
 
     val window1 = source
-      .windowAll(SlidingProcessingTimeWindows.of(
+      .windowAll(SlidingTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
       .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
@@ -161,12 +163,12 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val winOperator1 = operator1.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
     assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
     assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
     assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
 
 
     val window2 = source
-      .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+      .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
       .trigger(CountTrigger.of(100))
       .evictor(CountEvictor.of(1000))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
@@ -185,7 +187,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val winOperator2 = operator2.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
     assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
     assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
     assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
index 7232309..3f6e10f 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.scala
 
 import java.util.concurrent.TimeUnit
 
+import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.functions.TimestampExtractor
 import org.apache.flink.streaming.api.functions.sink.SinkFunction
 import org.apache.flink.streaming.api.functions.source.SourceFunction
@@ -38,9 +39,9 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
     CoGroupJoinITCase.testResults = mutable.MutableList()
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setParallelism(1)
-    env.getConfig.enableTimestamps
-    
+
     val source1 = env.addSource(new SourceFunction[(String, Int)]() {
       def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
         ctx.collect(("a", 0))
@@ -101,8 +102,8 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
     CoGroupJoinITCase.testResults = mutable.MutableList()
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setParallelism(1)
-    env.getConfig.enableTimestamps
 
     val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
       def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
@@ -177,8 +178,8 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
     CoGroupJoinITCase.testResults = mutable.MutableList()
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
     env.setParallelism(1)
-    env.getConfig.enableTimestamps
 
     val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
       def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {