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 2017/01/23 14:01:41 UTC

[01/10] flink git commit: [FLINK-4552] Refactor WindowOperator/Trigger Tests

Repository: flink
Updated Branches:
  refs/heads/release-1.2 f4cd7d0ec -> bb8586e50


http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index e682e2d..cc8f6d3 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -66,7 +66,6 @@ import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.streaming.util.WindowingTestHarness;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
@@ -2461,174 +2460,4 @@ public class WindowOperatorTest extends TestLogger {
 			return "EventTimeTrigger()";
 		}
 	}
-
-	@Test
-	public void testEventTimeTumblingWindowsWithOffset() throws Exception {
-		final int WINDOW_SIZE = 2000;
-		final int OFFSET = 100;
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE),Time.milliseconds(OFFSET));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			EventTimeTrigger.create(),
-			0);
-
-		// normal element
-		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
-		testHarness.processWatermark(1985);
-
-		testHarness.addExpectedWatermark(1985);
-
-		testHarness.processElement(new Tuple2<>("key2", 2), 1980);
-		testHarness.processElement(new Tuple2<>("key2", 3), 1998);
-		testHarness.processElement(new Tuple2<>("key2", 4), 2001);
-
-		// verify that this does not yet fire our windows, as it would without offsets
-		testHarness.processWatermark(2010);
-		testHarness.addExpectedWatermark(2010);
-
-		testHarness.processWatermark(2999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 1999 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 2), 1999 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 3), 1999 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 4), 1999 + OFFSET);
-
-		testHarness.addExpectedWatermark(2999);
-
-		testHarness.processWatermark(3999);
-		testHarness.addExpectedWatermark(3999);
-
-		testHarness.compareActualToExpectedOutput("Output is not correct");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testEventTimeSlidingWindowsWithOffset() throws Exception {
-		final int WINDOW_SIZE = 2000;
-		final int SLIDE = 500;
-		final int OFFSET = 10;
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		SlidingEventTimeWindows windowAssigner = SlidingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE),Time.milliseconds(SLIDE),Time.milliseconds(OFFSET));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			EventTimeTrigger.create(),
-			0);
-
-		testHarness.processElement(new Tuple2<>("key2", 1), 333);
-		testHarness.processWatermark(6666);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2",1),499 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2",1),999 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2",1),1499 + OFFSET);
-		testHarness.addExpectedElement(new Tuple2<>("key2",1),1999 + OFFSET);
-		testHarness.addExpectedWatermark(6666);
-		testHarness.compareActualToExpectedOutput("Output is not correct");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testProcessingTimeTumblingWindowsWithOffset() throws Exception {
-		final int WINDOW_SIZE = 3000;
-		final int OFFSET = 1000;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE),
-			Time.milliseconds(OFFSET));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			ProcessingTimeTrigger.create(),
-			0);
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new Tuple2<>("key2", 1), Long.MAX_VALUE);
-		testHarness.processElement(new Tuple2<>("key2", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key2", 1), 7000);
-
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-
-		testHarness.setProcessingTime(5000);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-
-		testHarness.setProcessingTime(7000);
-
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 6999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 6999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 6999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testProcessingTimeSlidingWindowsWithOffset() throws Exception {
-		final int WINDOW_SIZE = 3000;
-		final int SLIDING = 1000;
-		final int OFFSET = 10;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		SlidingProcessingTimeWindows windowAssigner = SlidingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE),
-			Time.milliseconds(SLIDING),Time.milliseconds(OFFSET));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			ProcessingTimeTrigger.create(),
-			0);
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new Tuple2<>("key2", 1), Long.MAX_VALUE);
-
-		testHarness.setProcessingTime(1111);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), OFFSET - 1);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), OFFSET + 999);
-
-		testHarness.processElement(new Tuple2<>("key2", 2),Long.MIN_VALUE);
-		testHarness.setProcessingTime(2222);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), OFFSET + 1999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 2), OFFSET + 1999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.close();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowedValue.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowedValue.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowedValue.java
new file mode 100644
index 0000000..449d54b
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowedValue.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+/**
+ * Helper class for emitting a value along with the window information from
+ * a {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction}.
+ */
+public class WindowedValue<T, W extends Window> {
+  private final T value;
+  private final W window;
+
+  public WindowedValue(T value, W window) {
+    this.value = value;
+    this.window = window;
+  }
+
+  public T value() {
+    return value;
+  }
+
+  public W window() {
+    return window;
+  }
+
+  @Override
+  public String toString() {
+    return "WindowedValue(" + value + ", " + window + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowingTestHarnessTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowingTestHarnessTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowingTestHarnessTest.java
deleted file mode 100644
index 82c3d71..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowingTestHarnessTest.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.WindowingTestHarness;
-import org.junit.Test;
-
-public class WindowingTestHarnessTest {
-
-	@Test
-	public void testEventTimeTumblingWindows() throws Exception {
-		final int WINDOW_SIZE = 2000;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			EventTimeTrigger.create(),
-			0);
-
-		// normal element
-		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
-		testHarness.processWatermark(1985);
-
-		testHarness.addExpectedWatermark(1985);
-
-		// this will not be dropped because window.maxTimestamp() + allowedLateness > currentWatermark
-		testHarness.processElement(new Tuple2<>("key2", 1), 1980);
-
-		// dropped as late
-		testHarness.processElement(new Tuple2<>("key2", 1), 1998);
-
-		testHarness.processElement(new Tuple2<>("key2", 1), 2001);
-		testHarness.processWatermark(2999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 1999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 1999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 1999);
-		testHarness.addExpectedWatermark(2999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 3999);
-
-		testHarness.processWatermark(3999);
-		testHarness.addExpectedWatermark(3999);
-
-		testHarness.compareActualToExpectedOutput("Output is not correct");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testProcessingTimeTumblingWindows() throws Exception {
-		final int WINDOW_SIZE = 3000;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			ProcessingTimeTrigger.create(),
-			0);
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new Tuple2<>("key2", 1), Long.MAX_VALUE);
-		testHarness.processElement(new Tuple2<>("key2", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key2", 1), 7000);
-
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-
-		testHarness.setProcessingTime(5000);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 2999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-		testHarness.processElement(new Tuple2<>("key1", 1), 7000);
-
-		testHarness.setProcessingTime(7000);
-
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 5999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 5999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 5999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testSnapshotingAndRecovery() throws Exception {
-
-		final int WINDOW_SIZE = 3000;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
-
-		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
-			windowAssigner,
-			BasicTypeInfo.STRING_TYPE_INFO,
-			inputType,
-			new TupleKeySelector(),
-			EventTimeTrigger.create(),
-			0);
-
-		// add elements out-of-order
-		testHarness.processElement(new Tuple2<>("key2", 1), 3999);
-		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
-
-		testHarness.processElement(new Tuple2<>("key1", 1), 20);
-		testHarness.processElement(new Tuple2<>("key1", 1), 0);
-		testHarness.processElement(new Tuple2<>("key1", 1), 999);
-
-		testHarness.processElement(new Tuple2<>("key2", 1), 1998);
-		testHarness.processElement(new Tuple2<>("key2", 1), 1999);
-		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
-
-		testHarness.processWatermark(999);
-		testHarness.addExpectedWatermark(999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processWatermark(1999);
-		testHarness.addExpectedWatermark(1999);
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		// do a snapshot, close and restore again
-		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.close();
-		testHarness.restore(snapshot);
-
-		testHarness.processWatermark(2999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key1", 1), 2999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 2999);
-
-		testHarness.addExpectedWatermark(2999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processWatermark(3999);
-		testHarness.addExpectedWatermark(3999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processWatermark(4999);
-		testHarness.addExpectedWatermark(4999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-		testHarness.processWatermark(5999);
-
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 5999);
-		testHarness.addExpectedElement(new Tuple2<>("key2", 1), 5999);
-		testHarness.addExpectedWatermark(5999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-
-
-		// those don't have any effect...
-		testHarness.processWatermark(6999);
-		testHarness.processWatermark(7999);
-
-		testHarness.addExpectedWatermark(6999);
-		testHarness.addExpectedWatermark(7999);
-
-		testHarness.compareActualToExpectedOutput("Output was not correct.");
-	}
-
-	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple2<String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
index 7fe4ebc..568410a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.util;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -46,6 +47,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperatorTest;
 import org.apache.flink.streaming.api.operators.OperatorSnapshotResult;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
@@ -554,6 +556,24 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 		return wasFailedExternally;
 	}
 
+	@VisibleForTesting
+	public int numProcessingTimeTimers() {
+		if (operator instanceof AbstractStreamOperator) {
+			return ((AbstractStreamOperator) operator).numProcessingTimeTimers();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@VisibleForTesting
+	public int numEventTimeTimers() {
+		if (operator instanceof AbstractStreamOperator) {
+			return ((AbstractStreamOperator) operator).numEventTimeTimers();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
 	private class MockOutput implements Output<StreamRecord<OUT>> {
 
 		private TypeSerializer<OUT> outputSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
index 4abb6e2..cde5780 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
@@ -29,6 +29,7 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -188,6 +189,22 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 		return false;
 	}
 
+	public int numKeyedStateEntries() {
+		if (keyedStateBackend instanceof HeapKeyedStateBackend) {
+			return ((HeapKeyedStateBackend) keyedStateBackend).numStateEntries();
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	public <N> int numKeyedStateEntries(N namespace) {
+		if (keyedStateBackend instanceof HeapKeyedStateBackend) {
+			return ((HeapKeyedStateBackend) keyedStateBackend).numStateEntries(namespace);
+		} else {
+			throw new UnsupportedOperationException();
+		}
+	}
+
 	@Override
 	public void initializeState(OperatorStateHandles operatorStateHandles) throws Exception {
 		if (operatorStateHandles != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
index 58e8c6b..4b6925d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
@@ -36,26 +36,13 @@ import static org.junit.Assert.assertEquals;
  * Utils for working with the various test harnesses.
  */
 public class TestHarnessUtil {
-	/**
-	 * Extracts the StreamRecords from the given output list.
-	 */
-	@SuppressWarnings("unchecked")
-	public static <OUT> List<StreamRecord<OUT>> getStreamRecordsFromOutput(List<Object> output) {
-		List<StreamRecord<OUT>> resultElements = new LinkedList<StreamRecord<OUT>>();
-		for (Object e: output) {
-			if (e instanceof StreamRecord) {
-				resultElements.add((StreamRecord<OUT>) e);
-			}
-		}
-		return resultElements;
-	}
 
 	/**
 	 * Extracts the raw elements from the given output list.
 	 */
 	@SuppressWarnings("unchecked")
 	public static <OUT> List<OUT> getRawElementsFromOutput(Queue<Object> output) {
-		List<OUT> resultElements = new LinkedList<OUT>();
+		List<OUT> resultElements = new LinkedList<>();
 		for (Object e: output) {
 			if (e instanceof StreamRecord) {
 				resultElements.add(((StreamRecord<OUT>) e).getValue());

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
deleted file mode 100644
index efb0d7e..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Comparator;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-/**
- * A utility class that facilitates the testing of custom {@link Trigger Triggers} and
- * {@link WindowAssigner WindowAssigners}.
- *
- * <p>For examples on how to use this class, see
- * {@link org.apache.flink.streaming.runtime.operators.windowing.WindowingTestHarnessTest}.
- *
- * <p>The input elements of type {@code IN} must implement the {@code equals()} method because
- * it is used to compare the expected output to the actual output.
- */
-public class WindowingTestHarness<K, IN, W extends Window> {
-
-	private final OneInputStreamOperatorTestHarness<IN, IN> testHarness;
-
-	private final ConcurrentLinkedQueue<Object> expectedOutputs = new ConcurrentLinkedQueue<>();
-
-	private volatile boolean isOpen = false;
-
-	public WindowingTestHarness(WindowAssigner<? super IN, W> windowAssigner,
-								TypeInformation<K> keyType,
-								TypeInformation<IN> inputType,
-								KeySelector<IN, K> keySelector,
-								Trigger<? super IN, ? super W> trigger,
-								long allowedLateness) throws Exception {
-
-		ListStateDescriptor<IN> windowStateDesc =
-				new ListStateDescriptor<>("window-contents", inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<K, IN, Iterable<IN>, IN, W> operator =
-			new WindowOperator<>(
-				windowAssigner,
-				windowAssigner.getWindowSerializer(new ExecutionConfig()),
-				keySelector,
-				keyType.createSerializer(new ExecutionConfig()),
-				windowStateDesc,
-				new InternalIterableWindowFunction<>(new PassThroughFunction()),
-				trigger,
-				allowedLateness);
-
-		testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, keyType);
-	}
-
-	/**
-	 * Simulates the processing of a new incoming element.
-	 */
-	public void processElement(IN element, long timestamp) throws Exception {
-		openOperator();
-		testHarness.processElement(new StreamRecord<>(element, timestamp));
-	}
-
-	/**
-	 * Simulates the processing of a new incoming watermark.
-	 */
-	public void processWatermark(long timestamp) throws Exception {
-		openOperator();
-		testHarness.processWatermark(new Watermark(timestamp));
-	}
-
-	/**
-	 * Sets the current processing time to {@code timestamp}.
-	 * This is useful when working on processing time.
-	 */
-	public void setProcessingTime(long timestamp) throws Exception {
-		openOperator();
-		testHarness.setProcessingTime(timestamp);
-	}
-
-	/**
-	 * Gets the current output of the windowing operator, as produced by the
-	 * synergies between the window assigner and the trigger. This will also
-	 * contain the received watermarks.
-	 */
-	public ConcurrentLinkedQueue<Object> getOutput() throws Exception {
-		return testHarness.getOutput();
-	}
-
-	/**
-	 * Closes the testing window operator.
-	 */
-	public void close() throws Exception {
-		if (isOpen) {
-			testHarness.close();
-			isOpen = false;
-		}
-	}
-
-	/**
-	 * Adds a watermark to the expected output.
-	 *
-	 * <p>The expected output should contain the elements and watermarks that we expect the output of the operator to
-	 * contain, in the correct order. This will be used to check if the produced output is the expected one, and
-	 * thus determine the success or failure of the test.
-	 */
-	public void addExpectedWatermark(long timestamp) {
-		expectedOutputs.add(new Watermark(timestamp));
-	}
-
-	/**
-	 * Adds an element to the expected output.
-	 *
-	 * <p>The expected output should contain the elements and watermarks that we expect the output of the operator to
-	 * contain, in the correct order. This will be used to check if the produced output is the expected one, and
-	 * thus determine the success or failure of the test.
-	 */
-	public void addExpectedElement(IN element, long timestamp) {
-		expectedOutputs.add(new StreamRecord<>(element, timestamp));
-	}
-
-	/**
-	 * Compares the current produced output with the expected one. The latter contains elements and watermarks added
-	 * using the {@link #addExpectedElement(Object, long)} and {@link #addExpectedWatermark(long)} methods.
-	 *
-	 * <p><b>NOTE:</b> This methods uses an {@code assert()} internally, thus failing the test if the {@code expected} output
-     * does not match the {@code actual} one.
-	 */
-	public void compareActualToExpectedOutput(String errorMessage) {
-		TestHarnessUtil.assertOutputEqualsSorted(errorMessage, expectedOutputs, testHarness.getOutput(), new StreamRecordComparator());
-	}
-
-	/**
-	 * Takes a snapshot of the current state of the operator. This can be used to test fault-tolerance.
-	 */
-	public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws Exception {
-		return testHarness.snapshot(checkpointId, timestamp);
-	}
-
-	/**
-	 * Resumes execution from the provided {@link OperatorStateHandles}. This is used to test recovery after a failure.
-	 */
-	public void restore(OperatorStateHandles stateHandles) throws Exception {
-		Preconditions.checkArgument(!isOpen,
-			"You are trying to restore() while the operator is still open. " +
-				"Please call close() first.");
-
-		testHarness.setup();
-		testHarness.initializeState(stateHandles);
-		openOperator();
-	}
-
-	private void openOperator() throws Exception {
-		if (!isOpen) {
-			testHarness.open();
-			isOpen = true;
-		}
-	}
-
-	private class PassThroughFunction implements WindowFunction<IN, IN, K, W> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void apply(K k, W window, Iterable<IN> input, Collector<IN> out) throws Exception {
-			for (IN in: input) {
-				out.collect(in);
-			}
-		}
-	}
-
-	/**
-	 * {@link Comparator} for sorting the expected and actual output by timestamp.
-	 */
-	@SuppressWarnings("unchecked")
-	private class StreamRecordComparator implements Comparator<Object> {
-		@Override
-		public int compare(Object o1, Object o2) {
-			if (o1 instanceof Watermark || o2 instanceof Watermark) {
-				return 0;
-			} else {
-				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
-				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
-				if (sr0.getTimestamp() != sr1.getTimestamp()) {
-					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
-				}
-				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
-				if (comparison != 0) {
-					return comparison;
-				} else {
-					return sr0.getValue().f1 - sr1.getValue().f1;
-				}
-			}
-		}
-	}
-}


[08/10] flink git commit: [hotfix] Use correct ClassLoader in FlinkKafkaConsumerBaseMigrationTest

Posted by al...@apache.org.
[hotfix] Use correct ClassLoader in FlinkKafkaConsumerBaseMigrationTest


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

Branch: refs/heads/release-1.2
Commit: 16b63c24ea5d34c16acaa71dc8b23cd87e82470f
Parents: 8fcd26b
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Jan 13 14:13:03 2017 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java     | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/16b63c24/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
index c315d31..fa93138 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
@@ -27,7 +27,6 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorTest;
 import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.util.SerializedValue;
@@ -60,7 +59,7 @@ import static org.mockito.Mockito.mock;
 public class FlinkKafkaConsumerBaseMigrationTest {
 
 	private static String getResourceFilename(String filename) {
-		ClassLoader cl = WindowOperatorTest.class.getClassLoader();
+		ClassLoader cl = FlinkKafkaConsumerBaseMigrationTest.class.getClassLoader();
 		URL resource = cl.getResource(filename);
 		if (resource == null) {
 			throw new NullPointerException("Missing snapshot resource.");


[02/10] flink git commit: [FLINK-4552] Refactor WindowOperator/Trigger Tests

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
new file mode 100644
index 0000000..47ead66
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -0,0 +1,2495 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.AppendingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.mockito.verification.VerificationMode;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.isStreamRecord;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+/**
+ * These tests verify that {@link WindowOperator} correctly interacts with the other windowing
+ * components: {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner},
+ * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} and window state.
+ *
+ * <p>These tests document the implicit contract that exists between the windowing components.
+ */
+public class WindowOperatorContractTest extends TestLogger {
+
+	private static ValueStateDescriptor<String> valueStateDescriptor =
+			new ValueStateDescriptor<>("string-state", StringSerializer.INSTANCE, null);
+
+	private static ListStateDescriptor<Integer> intListDescriptor =
+			new ListStateDescriptor<>("int-list", IntSerializer.INSTANCE);
+
+	private static ReducingStateDescriptor<Integer> intReduceSumDescriptor =
+			new ReducingStateDescriptor<>("int-reduce", new Sum(), IntSerializer.INSTANCE);
+
+	private static FoldingStateDescriptor<Integer, Integer> intFoldSumDescriptor =
+			new FoldingStateDescriptor<>("int-fold", 0, new FoldSum(), IntSerializer.INSTANCE);
+
+	static <IN, OUT, KEY, W extends Window> InternalWindowFunction<IN, OUT, KEY, W> mockWindowFunction() throws Exception {
+		@SuppressWarnings("unchecked")
+		InternalWindowFunction<IN, OUT, KEY, W> mockWindowFunction = mock(InternalWindowFunction.class);
+
+		return mockWindowFunction;
+	}
+
+	static <T, W extends Window> Trigger<T, W> mockTrigger() throws Exception {
+		@SuppressWarnings("unchecked")
+		Trigger<T, W> mockTrigger = mock(Trigger.class);
+
+		when(mockTrigger.onElement(Matchers.<T>any(), anyLong(), Matchers.<W>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		when(mockTrigger.onEventTime(anyLong(), Matchers.<W>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<W>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+
+		return mockTrigger;
+	}
+
+	static <T> WindowAssigner<T, TimeWindow> mockTimeWindowAssigner() throws Exception {
+		@SuppressWarnings("unchecked")
+		WindowAssigner<T, TimeWindow> mockAssigner = mock(WindowAssigner.class);
+
+		when(mockAssigner.getWindowSerializer(Mockito.<ExecutionConfig>any())).thenReturn(new TimeWindow.Serializer());
+		when(mockAssigner.isEventTime()).thenReturn(true);
+
+		return mockAssigner;
+	}
+
+	static <T> WindowAssigner<T, GlobalWindow> mockGlobalWindowAssigner() throws Exception {
+		@SuppressWarnings("unchecked")
+		WindowAssigner<T, GlobalWindow> mockAssigner = mock(WindowAssigner.class);
+
+		when(mockAssigner.getWindowSerializer(Mockito.<ExecutionConfig>any())).thenReturn(new GlobalWindow.Serializer());
+		when(mockAssigner.isEventTime()).thenReturn(true);
+		when(mockAssigner.assignWindows(Mockito.<T>any(), anyLong(), anyAssignerContext())).thenReturn(Collections.singletonList(GlobalWindow.get()));
+
+		return mockAssigner;
+	}
+
+
+	static <T> MergingWindowAssigner<T, TimeWindow> mockMergingAssigner() throws Exception {
+		@SuppressWarnings("unchecked")
+		MergingWindowAssigner<T, TimeWindow> mockAssigner = mock(MergingWindowAssigner.class);
+
+		when(mockAssigner.getWindowSerializer(Mockito.<ExecutionConfig>any())).thenReturn(new TimeWindow.Serializer());
+		when(mockAssigner.isEventTime()).thenReturn(true);
+
+		return mockAssigner;
+	}
+
+
+	static WindowAssigner.WindowAssignerContext anyAssignerContext() {
+		return Mockito.any();
+	}
+
+	static Trigger.TriggerContext anyTriggerContext() {
+		return Mockito.any();
+	}
+
+	static <T> Collector<T> anyCollector() {
+		return Mockito.any();
+	}
+
+	static Iterable<Integer> anyIntIterable() {
+		return Mockito.any();
+	}
+
+	@SuppressWarnings("unchecked")
+	static Iterable<Integer> intIterable(Integer... values) {
+		return (Iterable<Integer>) argThat(containsInAnyOrder(values));
+	}
+
+	static TimeWindow anyTimeWindow() {
+		return Mockito.any();
+	}
+
+	static Trigger.OnMergeContext anyOnMergeContext() {
+		return Mockito.any();
+	}
+
+	static MergingWindowAssigner.MergeCallback anyMergeCallback() {
+		return Mockito.any();
+	}
+
+
+	static <T> void shouldRegisterEventTimeTimerOnElement(Trigger<T, TimeWindow> mockTrigger, final long timestamp) throws Exception {
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Trigger.TriggerContext context =
+						(Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(timestamp);
+				return TriggerResult.CONTINUE;
+			}
+		})
+		.when(mockTrigger).onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+	}
+
+	private static <T> void shouldDeleteEventTimeTimerOnElement(Trigger<T, TimeWindow> mockTrigger, final long timestamp) throws Exception {
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Trigger.TriggerContext context =
+						(Trigger.TriggerContext) invocation.getArguments()[3];
+				context.deleteEventTimeTimer(timestamp);
+				return TriggerResult.CONTINUE;
+			}
+		})
+		.when(mockTrigger).onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+	}
+
+	private static <T> void shouldRegisterProcessingTimeTimerOnElement(Trigger<T, TimeWindow> mockTrigger, final long timestamp) throws Exception {
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Trigger.TriggerContext context =
+						(Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerProcessingTimeTimer(timestamp);
+				return TriggerResult.CONTINUE;
+			}
+		})
+				.when(mockTrigger).onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+	}
+
+	private static <T> void shouldDeleteProcessingTimeTimerOnElement(Trigger<T, TimeWindow> mockTrigger, final long timestamp) throws Exception {
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Trigger.TriggerContext context =
+						(Trigger.TriggerContext) invocation.getArguments()[3];
+				context.deleteProcessingTimeTimer(timestamp);
+				return TriggerResult.CONTINUE;
+			}
+		})
+				.when(mockTrigger).onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+	}
+
+	@SuppressWarnings("unchecked")
+	private static <T, W extends Window> void shouldMergeWindows(final MergingWindowAssigner<T, W> assigner, final Collection<? extends W> expectedWindows, final Collection<W> toMerge, final W mergeResult) {
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Collection<W> windows = (Collection<W>) invocation.getArguments()[0];
+
+				MergingWindowAssigner.MergeCallback callback = (MergingWindowAssigner.MergeCallback) invocation.getArguments()[1];
+
+				// verify the expected windows
+				assertThat(windows, containsInAnyOrder(expectedWindows.toArray()));
+
+				callback.merge(toMerge, mergeResult);
+				return null;
+			}
+		})
+				.when(assigner).mergeWindows(anyCollection(), Matchers.<MergingWindowAssigner.MergeCallback>anyObject());
+	}
+
+	private static <T> void shouldContinueOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+	}
+
+	private static <T> void shouldFireOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+	}
+
+	private static <T> void shouldPurgeOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+	}
+
+	private static <T> void shouldFireAndPurgeOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+	}
+
+	private static <T> void shouldContinueOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+	}
+
+	private static <T> void shouldFireOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+	}
+
+	private static <T> void shouldPurgeOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+	}
+
+	private static <T> void shouldFireAndPurgeOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+	}
+
+	private static <T> void shouldContinueOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+	}
+
+	private static <T> void shouldFireOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+	}
+
+	private static <T> void shouldPurgeOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+	}
+
+	private static <T> void shouldFireAndPurgeOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
+		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+	}
+
+	@Test
+	public void testAssignerIsInvokedOncePerElement() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 0)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockAssigner, times(1)).assignWindows(eq(0), eq(0L), anyAssignerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockAssigner, times(2)).assignWindows(eq(0), eq(0L), anyAssignerContext());
+
+	}
+
+	@Test
+	public void testAssignerWithMultipleWindows() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		shouldFireOnElement(mockTrigger);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+	}
+
+	@Test
+	public void testWindowsDontInterfere() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 2)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 1)));
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		// no output so far
+		assertTrue(testHarness.extractOutputStreamRecords().isEmpty());
+
+		// state for two windows
+		assertEquals(2, testHarness.numKeyedStateEntries());
+		assertEquals(2, testHarness.numEventTimeTimers());
+
+		// now we fire
+		shouldFireOnElement(mockTrigger);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 1)));
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 2)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0, 0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 1)), intIterable(1, 1), WindowOperatorContractTest.<Void>anyCollector());
+	}
+
+	@Test
+	public void testOnElementCalledPerWindow() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		testHarness.processElement(new StreamRecord<>(42, 1L));
+
+		verify(mockTrigger).onElement(eq(42), eq(1L), eq(new TimeWindow(2, 4)), anyTriggerContext());
+		verify(mockTrigger).onElement(eq(42), eq(1L), eq(new TimeWindow(0, 2)), anyTriggerContext());
+
+		verify(mockTrigger, times(2)).onElement(anyInt(), anyLong(), anyTimeWindow(), anyTriggerContext());
+	}
+
+	@Test
+	public void testReducingWindow() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intReduceSumDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		// insert two elements without firing
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+	}
+
+	@Test
+	public void testFoldingWindow() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intFoldSumDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		// insert two elements without firing
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+	}
+
+	@Test
+	public void testEmittingFromWindowFunction() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, String, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, String> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 2)));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Collector<String> out = invocation.getArgumentAt(3, Collector.class);
+				out.collect("Hallo");
+				out.collect("Ciao");
+				return null;
+			}
+		}).when(mockWindowFunction).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+
+		assertThat(testHarness.extractOutputStreamRecords(),
+				containsInAnyOrder(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
+	}
+
+	@Test
+	public void testEmittingFromWindowFunctionOnEventTime() throws Exception {
+		testEmittingFromWindowFunction(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testEmittingFromWindowFunctionOnProcessingTime() throws Exception {
+		testEmittingFromWindowFunction(new ProcessingTimeAdaptor());
+	}
+
+
+	private  void testEmittingFromWindowFunction(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, String, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, String> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Collections.singletonList(new TimeWindow(0, 2)));
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Exception {
+				@SuppressWarnings("unchecked")
+				Collector<String> out = invocation.getArgumentAt(3, Collector.class);
+				out.collect("Hallo");
+				out.collect("Ciao");
+				return null;
+			}
+		}).when(mockWindowFunction).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 1);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, never()).apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<String>anyCollector());
+		assertTrue(testHarness.extractOutputStreamRecords().isEmpty());
+
+		timeAdaptor.shouldFireOnTime(mockTrigger);
+
+		timeAdaptor.advanceTime(testHarness, 1L);
+
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+
+		assertThat(testHarness.extractOutputStreamRecords(),
+				containsInAnyOrder(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
+	}
+
+	@Test
+	public void testOnElementContinue() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// CONTINUE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+
+		// there should be no firing
+		assertEquals(0, testHarness.getOutput().size());
+	}
+
+	@Test
+	public void testOnElementFire() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+	}
+
+	@Test
+	public void testOnElementFireAndPurge() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE_AND_PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE_AND_PURGE should purge contents
+		assertEquals(2, testHarness.numKeyedStateEntries()); // trigger state will stick around until GC time
+
+		// timers will stick around
+		assertEquals(4, testHarness.numEventTimeTimers());
+	}
+
+	@Test
+	public void testOnElementPurge() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// PURGE should purge contents
+		assertEquals(2, testHarness.numKeyedStateEntries()); // trigger state will stick around until GC time
+
+		// timers will stick around
+		assertEquals(4, testHarness.numEventTimeTimers()); // trigger timer and GC timer
+
+		// no output
+		assertEquals(0, testHarness.getOutput().size());
+	}
+
+	@Test
+	public void testOnEventTimeContinue() throws Exception {
+		testOnTimeContinue(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testOnProcessingTimeContinue() throws Exception {
+		testOnTimeContinue(new ProcessingTimeAdaptor());
+	}
+
+	private void testOnTimeContinue(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		// this should register two timers because we have two windows
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// we don't want to fire the cleanup timer
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.shouldContinueOnTime(mockTrigger);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window-contents plus trigger state for two windows
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // timers/gc timers for two windows
+
+		timeAdaptor.advanceTime(testHarness, 0L);
+
+		assertEquals(4, testHarness.numKeyedStateEntries());
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // only gc timers left
+
+		// there should be no firing
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+	}
+
+	@Test
+	public void testOnEventTimeFire() throws Exception {
+		testOnTimeFire(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testOnProcessingTimeFire() throws Exception {
+		testOnTimeFire(new ProcessingTimeAdaptor());
+	}
+
+	private void testOnTimeFire(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.shouldFireOnTime(mockTrigger);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window-contents and trigger state for two windows
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // timers/gc timers for two windows
+
+		timeAdaptor.advanceTime(testHarness, 0L);
+
+		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries());
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // only gc timers left
+	}
+
+	@Test
+	public void testOnEventTimeFireAndPurge() throws Exception {
+		testOnTimeFireAndPurge(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testOnProcessingTimeFireAndPurge() throws Exception {
+		testOnTimeFireAndPurge(new ProcessingTimeAdaptor());
+	}
+
+	private void testOnTimeFireAndPurge(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.shouldFireAndPurgeOnTime(mockTrigger);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window-contents and trigger state for two windows
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // timers/gc timers for two windows
+
+		timeAdaptor.advanceTime(testHarness, 0L);
+
+		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE_AND_PURGE should purge contents
+		assertEquals(2, testHarness.numKeyedStateEntries()); // trigger state stays until GC time
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // gc timers are still there
+	}
+
+	@Test
+	public void testOnEventTimePurge() throws Exception {
+		testOnTimePurge(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testOnProcessingTimePurge() throws Exception {
+		testOnTimePurge(new ProcessingTimeAdaptor());
+	}
+
+	private void testOnTimePurge(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(4, 6)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 1L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.shouldPurgeOnTime(mockTrigger);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window-contents and trigger state for two windows
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // timers/gc timers for two windows
+
+		timeAdaptor.advanceTime(testHarness, 1L);
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// PURGE should purge contents
+		assertEquals(2, testHarness.numKeyedStateEntries()); // trigger state will stick around
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // gc timers are still there
+
+		// still no output
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+	}
+
+	@Test
+	public void testNoEventTimeFiringForPurgedWindow() throws Exception {
+		testNoTimerFiringForPurgedWindow(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testNoProcessingTimeFiringForPurgedWindow() throws Exception {
+		testNoTimerFiringForPurgedWindow(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * Verify that we neither invoke the trigger nor the window function if a timer
+	 * for a non-existent window fires.
+	 */
+	private void testNoTimerFiringForPurgedWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+
+		@SuppressWarnings("unchecked")
+		InternalWindowFunction<Iterable<Integer>, List<Integer>, Integer, TimeWindow> mockWindowFunction =
+				mock(InternalWindowFunction.class);
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, List<Integer>> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(0, testHarness.numKeyedStateEntries()); // not contents or state
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // timer and gc timer
+
+		timeAdaptor.advanceTime(testHarness, 0L);
+
+		// trigger is not called if there is no more window (timer is silently ignored)
+		timeAdaptor.verifyTriggerCallback(mockTrigger, never(), null, null);
+
+		verify(mockWindowFunction, never())
+				.apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // only gc timers left
+	}
+
+	@Test
+	public void testNoEventTimeFiringForPurgedMergingWindow() throws Exception {
+		testNoTimerFiringForPurgedMergingWindow(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testNoProcessingTimeFiringForPurgedMergingWindow() throws Exception {
+		testNoTimerFiringForPurgedMergingWindow(new ProcessingTimeAdaptor());
+	}
+
+
+	/**
+	 * Verify that we neither invoke the trigger nor the window function if a timer
+	 * for a non-existent merging window fires.
+	 */
+	public void testNoTimerFiringForPurgedMergingWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+
+		@SuppressWarnings("unchecked")
+		InternalWindowFunction<Iterable<Integer>, List<Integer>, Integer, TimeWindow> mockWindowFunction =
+				mock(InternalWindowFunction.class);
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, List<Integer>> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(1, testHarness.numKeyedStateEntries()); // just the merging window set
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // timer and gc timer
+
+		timeAdaptor.advanceTime(testHarness, 0L);
+
+		// trigger is not called if there is no more window (timer is silently ignored)
+		timeAdaptor.verifyTriggerCallback(mockTrigger, never(), null, null);
+
+		verify(mockWindowFunction, never())
+				.apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // only gc timers left
+	}
+
+	@Test
+	public void testEventTimeTimerCreationAndDeletion() throws Exception {
+		testTimerCreationAndDeletion(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeTimerCreationAndDeletion() throws Exception {
+		testTimerCreationAndDeletion(new ProcessingTimeAdaptor());
+	}
+
+	private void testTimerCreationAndDeletion(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 2)));
+
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 17);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 42);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(3, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+
+		timeAdaptor.shouldDeleteTimerOnElement(mockTrigger, 42);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		timeAdaptor.shouldDeleteTimerOnElement(mockTrigger, 17);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+	}
+
+	@Test
+	public void testEventTimeTimerFiring() throws Exception {
+		testTimerFiring(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeTimerFiring() throws Exception {
+		testTimerFiring(new ProcessingTimeAdaptor());
+	}
+
+
+	private void testTimerFiring(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 100)));
+
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 1);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 17);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 42);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+
+		timeAdaptor.advanceTime(testHarness, 1);
+
+		timeAdaptor.verifyTriggerCallback(mockTrigger, atLeastOnce(), 1L, new TimeWindow(0, 100));
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), null, null);
+		assertEquals(3, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+
+		// doesn't do anything
+		timeAdaptor.advanceTime(testHarness, 15);
+
+		// so still the same
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), null, null);
+
+		timeAdaptor.advanceTime(testHarness, 42);
+
+		timeAdaptor.verifyTriggerCallback(mockTrigger, atLeastOnce(), 17L, new TimeWindow(0, 100));
+		timeAdaptor.verifyTriggerCallback(mockTrigger, atLeastOnce(), 42L, new TimeWindow(0, 100));
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(3), null, null);
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // +1 because of the GC timer of the window
+	}
+
+	@Test
+	public void testEventTimeDeletedTimerDoesNotFire() throws Exception {
+		testDeletedTimerDoesNotFire(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeDeletedTimerDoesNotFire() throws Exception {
+		testDeletedTimerDoesNotFire(new ProcessingTimeAdaptor());
+	}
+
+	public void testDeletedTimerDoesNotFire(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 100)));
+
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 1);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // +1 for the GC timer
+
+		timeAdaptor.shouldDeleteTimerOnElement(mockTrigger, 1);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // +1 for the GC timer
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 2);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // +1 for the GC timer
+
+		timeAdaptor.advanceTime(testHarness, 50L);
+
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(0), 1L, null);
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), 2L, new TimeWindow(0, 100));
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), null, null);
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // +1 for the GC timer
+	}
+
+	@Test
+	public void testMergeWindowsIsCalled() throws Exception {
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockAssigner).mergeWindows(eq(Lists.newArrayList(new TimeWindow(2, 4))), anyMergeCallback());
+		verify(mockAssigner).mergeWindows(eq(Lists.newArrayList(new TimeWindow(2, 4), new TimeWindow(0, 2))), anyMergeCallback());
+		verify(mockAssigner, times(2)).mergeWindows(anyCollection(), anyMergeCallback());
+
+
+	}
+
+	@Test
+	public void testEventTimeWindowsAreMergedEagerly() throws Exception {
+		testWindowsAreMergedEagerly(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeWindowsAreMergedEagerly() throws Exception {
+		testWindowsAreMergedEagerly(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * Verify that windows are merged eagerly, if possible.
+	 */
+	public void testWindowsAreMergedEagerly(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		// in this test we only have one state window and windows are eagerly
+		// merged into the first window
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't intefere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.OnMergeContext context = (Trigger.OnMergeContext) invocation.getArguments()[1];
+				// don't intefere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onMerge(anyTimeWindow(), anyOnMergeContext());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[1];
+				timeAdaptor.deleteTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 2)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(3, testHarness.numKeyedStateEntries()); // window state plus trigger state plus merging window set
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // timer and GC timer
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4)));
+
+		shouldMergeWindows(
+				mockAssigner,
+				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)),
+				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)),
+				new TimeWindow(0, 4));
+
+		// don't register a timer or update state in onElement, this checks
+		// whether onMerge does correctly set those things
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		verify(mockTrigger).onMerge(eq(new TimeWindow(0, 4)), anyOnMergeContext());
+
+		assertEquals(3, testHarness.numKeyedStateEntries());
+		assertEquals(2, timeAdaptor.numTimers(testHarness));
+	}
+
+	@Test
+	public void testMergingOfExistingEventTimeWindows() throws Exception {
+		testMergingOfExistingWindows(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testMergingOfExistingProcessingTimeWindows() throws Exception {
+		testMergingOfExistingWindows(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * Verify that we only keep one of the underlying state windows. This test also verifies that
+	 * GC timers are correctly deleted when merging windows.
+	 */
+	public void testMergingOfExistingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, Long.MIN_VALUE);
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.OnMergeContext context = (Trigger.OnMergeContext) invocation.getArguments()[1];
+				// don't interfere with cleanup timers
+				timeAdaptor.registerTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onMerge(anyTimeWindow(), anyOnMergeContext());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[1];
+				// don't interfere with cleanup timers
+				timeAdaptor.deleteTimer(context, 0L);
+				context.getPartitionedState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 2)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(3, testHarness.numKeyedStateEntries()); // window state plus trigger state plus merging window set
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // trigger timer plus GC timer
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(5, testHarness.numKeyedStateEntries()); // window state plus trigger state plus merging window set
+		assertEquals(4, timeAdaptor.numTimers(testHarness)); // trigger timer plus GC timer
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(1, 3)));
+
+		shouldMergeWindows(
+				mockAssigner,
+				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3)),
+				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3)),
+				new TimeWindow(0, 4));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(3, testHarness.numKeyedStateEntries()); // window contents plus trigger state plus merging window set
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // trigger timer plus GC timer
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+	}
+
+	@Test
+	public void testOnElementPurgeDoesNotCleanupMergingSet() throws Exception {
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(1, testHarness.numKeyedStateEntries()); // the merging window set
+
+		assertEquals(1, testHarness.numEventTimeTimers()); // one cleanup timer
+
+		assertEquals(0, testHarness.getOutput().size());
+	}
+
+	@Test
+	public void testOnEventTimePurgeDoesNotCleanupMergingSet() throws Exception {
+		testOnTimePurgeDoesNotCleanupMergingSet(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testOnProcessingTimePurgeDoesNotCleanupMergingSet() throws Exception {
+		testOnTimePurgeDoesNotCleanupMergingSet(new ProcessingTimeAdaptor());
+	}
+
+	public void testOnTimePurgeDoesNotCleanupMergingSet(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 4)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 1L);
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		timeAdaptor.shouldPurgeOnTime(mockTrigger);
+
+		assertEquals(2, testHarness.numKeyedStateEntries()); // the merging window set + window contents
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // one cleanup timer + timer
+		assertEquals(0, testHarness.getOutput().size());
+
+		timeAdaptor.advanceTime(testHarness, 1L);
+
+		assertEquals(1, testHarness.numKeyedStateEntries()); // the merging window set
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // one cleanup timer
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+	}
+
+	@Test
+	public void testNoEventTimeGarbageCollectionTimerForGlobalWindow() throws Exception {
+		testNoGarbageCollectionTimerForGlobalWindow(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testNoProcessingTimeGarbageCollectionTimerForGlobalWindow() throws Exception {
+		testNoGarbageCollectionTimerForGlobalWindow(new ProcessingTimeAdaptor());
+	}
+
+	public void testNoGarbageCollectionTimerForGlobalWindow(TimeDomainAdaptor timeAdaptor) throws Exception {
+
+		WindowAssigner<Integer, GlobalWindow> mockAssigner = mockGlobalWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, GlobalWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, GlobalWindow> mockWindowFunction = mockWindowFunction();
+
+		// this needs to be true for the test to succeed
+		assertEquals(Long.MAX_VALUE, GlobalWindow.get().maxTimestamp());
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// just the window contents
+		assertEquals(1, testHarness.numKeyedStateEntries());
+
+		// verify we have no timers for either time domain
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+	}
+
+	@Test
+	public void testNoEventTimeGarbageCollectionTimerForLongMax() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, Long.MAX_VALUE - 10)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// just the window contents
+		assertEquals(1, testHarness.numKeyedStateEntries());
+
+		// no GC timer
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+	}
+
+	@Test
+	public void testProcessingTimeGarbageCollectionTimerIsAlwaysWindowMaxTimestamp() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		when(mockAssigner.isEventTime()).thenReturn(false);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, Long.MAX_VALUE - 10)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// just the window contents
+		assertEquals(1, testHarness.numKeyedStateEntries());
+
+		// no GC timer
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers());
+
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		testHarness.setProcessingTime(Long.MAX_VALUE - 10);
+
+		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+	}
+
+	@Test
+	public void testEventTimeGarbageCollectionTimer() throws Exception {
+		testGarbageCollectionTimer(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeGarbageCollectionTimer() throws Exception {
+		testGarbageCollectionTimer(new ProcessingTimeAdaptor());
+	}
+
+	public void testGarbageCollectionTimer(TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// just the window contents
+		assertEquals(1, testHarness.numKeyedStateEntries());
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness));
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.advanceTime(testHarness, 19 + 20); // 19 is maxTime of the window
+
+		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+	}
+
+	@Test
+	public void testEventTimeTriggerTimerAndGarbageCollectionTimerCoincide() throws Exception {
+		testTriggerTimerAndGarbageCollectionTimerCoincide(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testProcessingTimeTriggerTimerAndGarbageCollectionTimerCoincide() throws Exception {
+		testTriggerTimerAndGarbageCollectionTimerCoincide(new ProcessingTimeAdaptor());
+	}
+
+	public void testTriggerTimerAndGarbageCollectionTimerCoincide(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// 19 is maxTime of window
+				timeAdaptor.registerTimer(context, 19L);
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// just the window contents
+		assertEquals(1, testHarness.numKeyedStateEntries());
+
+		assertEquals(1, timeAdaptor.numTimers(testHarness));
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		timeAdaptor.advanceTime(testHarness, 19); // 19 is maxTime of the window
+
+		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), null, null);
+
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtEventTimeGarbageCollection() throws Exception {
+		testStateAndTimerCleanupAtEventTimeGarbageCollection(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtProcessingTimeGarbageCollection() throws Exception {
+		testStateAndTimerCleanupAtEventTimeGarbageCollection(new ProcessingTimeAdaptor());
+	}
+
+	public void testStateAndTimerCleanupAtEventTimeGarbageCollection(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// very far in the future so our watermark does not trigger it
+				timeAdaptor.registerTimer(context, 1000L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[1];
+				timeAdaptor.deleteTimer(context, 1000L);
+				context.getPartitionedState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(2, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // window timers/gc timers
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+
+		timeAdaptor.advanceTime(testHarness, 19 + 20);
+
+		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(0, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(0, timeAdaptor.numTimers(testHarness)); // window timers/gc timers
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtEventTimeGarbageCollectionWithPurgingTrigger() throws Exception {
+		testStateAndTimerCleanupAtEventTimeGCWithPurgingTrigger(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtProcessingTimeGarbageCollectionWithPurgingTrigger() throws Exception {
+		testStateAndTimerCleanupAtEventTimeGCWithPurgingTrigger(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * Verify that we correctly clean up even when a purging trigger has purged
+	 * window state.
+	 */
+	public void testStateAndTimerCleanupAtEventTimeGCWithPurgingTrigger(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// very far in the future so our watermark does not trigger it
+				timeAdaptor.registerTimer(context, 1000L);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[1];
+				timeAdaptor.deleteTimer(context, 1000L);
+				context.getPartitionedState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(1, testHarness.numKeyedStateEntries()); // just the trigger state remains
+		assertEquals(2, timeAdaptor.numTimers(testHarness)); // window timers/gc timers
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+
+		timeAdaptor.advanceTime(testHarness, 19 + 20); // 19 is maxTime of the window
+
+		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(0, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(0, timeAdaptor.numTimers(testHarness)); // window timers/gc timers
+		assertEquals(0, timeAdaptor.numTimersOtherDomain(testHarness));
+
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtEventTimeGarbageCollectionWithPurgingTriggerAndMergingWindows() throws Exception {
+		testStateAndTimerCleanupAtGarbageCollectionWithPurgingTriggerAndMergingWindows(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testStateAndTimerCleanupAtProcessingTimeGarbageCollectionWithPurgingTriggerAndMergingWindows() throws Exception {
+		testStateAndTimerCleanupAtGarbageCollectionWithPurgingTriggerAndMergingWindows(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * Verify that we correctly clean up even when a purging trigger has purged
+	 * window state.
+	 */
+	public void testStateAndTimerCleanupAtGarbageCollectionWithPurgingTriggerAndMergingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				// very far in the future so our watermark does not trigger it
+				timeAdaptor.registerTimer(context, 1000);
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.PURGE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[1];
+				timeAdaptor.deleteTimer(context, 1000);
+				context.getPartitionedState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		assertEquals(2, testHarness.numKey

<TRUNCATED>

[06/10] flink git commit: [hotfix] Use own SumReducer in WindowOperatorMigrationTest

Posted by al...@apache.org.
[hotfix] Use own SumReducer in WindowOperatorMigrationTest


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

Branch: refs/heads/release-1.2
Commit: 8fcd26b7e881cf4901d30cd37c4de8925b59e985
Parents: e3bab4a
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Jan 12 18:54:38 2017 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../operators/windowing/WindowOperatorMigrationTest.java     | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8fcd26b7/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
index b7d5928..429ffbe 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
@@ -424,7 +424,7 @@ public class WindowOperatorMigrationTest {
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
 
 		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new WindowOperatorTest.SumReducer(),
+				new SumReducer(),
 				inputType.createSerializer(new ExecutionConfig()));
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
@@ -508,7 +508,7 @@ public class WindowOperatorMigrationTest {
 				new TupleKeySelector(),
 				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
 				stateDesc,
-				new InternalIterableWindowFunction<>(new WindowOperatorTest.RichSumReducer<TimeWindow>()),
+				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
 				ProcessingTimeTrigger.create(),
 				0);
 
@@ -621,7 +621,7 @@ public class WindowOperatorMigrationTest {
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
 
 		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new WindowOperatorTest.SumReducer(),
+				new SumReducer(),
 				inputType.createSerializer(new ExecutionConfig()));
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
@@ -723,7 +723,7 @@ public class WindowOperatorMigrationTest {
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
 
 		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new WindowOperatorTest.SumReducer(),
+				new SumReducer(),
 				inputType.createSerializer(new ExecutionConfig()));
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(


[03/10] flink git commit: [FLINK-4552] Refactor WindowOperator/Trigger Tests

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SimpleTriggerTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SimpleTriggerTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SimpleTriggerTestHarness.java
new file mode 100644
index 0000000..050178b
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SimpleTriggerTestHarness.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+/**
+ * Simple {@link TriggerTestHarness} that accepts integers and takes the value as the timestamp for
+ * the {@link StreamRecord}.
+ */
+public class SimpleTriggerTestHarness<W extends Window> extends TriggerTestHarness<Integer, W> {
+
+	public SimpleTriggerTestHarness(
+			Trigger<Integer, W> trigger,
+			TypeSerializer<W> windowSerializer) throws Exception {
+		super(trigger, windowSerializer);
+	}
+
+	public TriggerResult processElement(Integer element, W window) throws Exception {
+		return super.processElement(new StreamRecord<>(element, element), window);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingEventTimeWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingEventTimeWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingEventTimeWindowsTest.java
new file mode 100644
index 0000000..4599d19
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingEventTimeWindowsTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for {@link SlidingEventTimeWindows}
+ */
+public class SlidingEventTimeWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingEventTimeWindows assigner =
+				SlidingEventTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(1000));
+
+		assertThat(assigner.assignWindows("String", 0L, mockContext), containsInAnyOrder(
+				timeWindow(-4000, 1000),
+				timeWindow(-3000, 2000),
+				timeWindow(-2000, 3000),
+				timeWindow(-1000, 4000),
+				timeWindow(0, 5000)));
+
+		assertThat(assigner.assignWindows("String", 4999L, mockContext), containsInAnyOrder(
+				timeWindow(0, 5000),
+				timeWindow(1000, 6000),
+				timeWindow(2000, 7000),
+				timeWindow(3000, 8000),
+				timeWindow(4000, 9000)));
+
+		assertThat(assigner.assignWindows("String", 5000L, mockContext), containsInAnyOrder(
+				timeWindow(1000, 6000),
+				timeWindow(2000, 7000),
+				timeWindow(3000, 8000),
+				timeWindow(4000, 9000),
+				timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testWindowAssignmentWithOffset() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingEventTimeWindows assigner =
+				SlidingEventTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(1000), Time.milliseconds(100));
+
+		assertThat(assigner.assignWindows("String", 100L, mockContext), containsInAnyOrder(
+				timeWindow(-3900, 1100),
+				timeWindow(-2900, 2100),
+				timeWindow(-1900, 3100),
+				timeWindow(-900, 4100),
+				timeWindow(100, 5100)));
+
+		assertThat(assigner.assignWindows("String", 5099L, mockContext), containsInAnyOrder(
+				timeWindow(100, 5100),
+				timeWindow(1100, 6100),
+				timeWindow(2100, 7100),
+				timeWindow(3100, 8100),
+				timeWindow(4100, 9100)));
+
+		assertThat(assigner.assignWindows("String", 5100L, mockContext), containsInAnyOrder(
+				timeWindow(1100, 6100),
+				timeWindow(2100, 7100),
+				timeWindow(3100, 8100),
+				timeWindow(4100, 9100),
+				timeWindow(5100, 10100)));
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingEventTimeWindows assigner = SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1), Time.milliseconds(500));
+
+		assertThat(assigner.assignWindows("String", 100L, mockContext), containsInAnyOrder(
+				timeWindow(-4500, 500),
+				timeWindow(-3500, 1500),
+				timeWindow(-2500, 2500),
+				timeWindow(-1500, 3500),
+				timeWindow(-500, 4500)));
+
+		assertThat(assigner.assignWindows("String", 5499L, mockContext), containsInAnyOrder(
+				timeWindow(500, 5500),
+				timeWindow(1500, 6500),
+				timeWindow(2500, 7500),
+				timeWindow(3500, 8500),
+				timeWindow(4500, 9500)));
+
+		assertThat(assigner.assignWindows("String", 5100L, mockContext), containsInAnyOrder(
+				timeWindow(500, 5500),
+				timeWindow(1500, 6500),
+				timeWindow(2500, 7500),
+				timeWindow(3500, 8500),
+				timeWindow(4500, 9500)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			SlidingEventTimeWindows.of(Time.seconds(-2), Time.seconds(1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+
+		try {
+			SlidingEventTimeWindows.of(Time.seconds(2), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+
+
+		try {
+			SlidingEventTimeWindows.of(Time.seconds(20), Time.seconds(10), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+	}
+
+	@Test
+	public void testProperties() {
+		SlidingEventTimeWindows assigner = SlidingEventTimeWindows.of(Time.seconds(5), Time.milliseconds(100));
+
+		assertTrue(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(EventTimeTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingProcessingTimeWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingProcessingTimeWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingProcessingTimeWindowsTest.java
new file mode 100644
index 0000000..20a9924
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/SlidingProcessingTimeWindowsTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link SlidingProcessingTimeWindows}
+ */
+public class SlidingProcessingTimeWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingProcessingTimeWindows assigner =
+				SlidingProcessingTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(1000));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(0L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(-4000, 1000),
+				timeWindow(-3000, 2000),
+				timeWindow(-2000, 3000),
+				timeWindow(-1000, 4000),
+				timeWindow(0, 5000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(4999L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(0, 5000),
+				timeWindow(1000, 6000),
+				timeWindow(2000, 7000),
+				timeWindow(3000, 8000),
+				timeWindow(4000, 9000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(1000, 6000),
+				timeWindow(2000, 7000),
+				timeWindow(3000, 8000),
+				timeWindow(4000, 9000),
+				timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testWindowAssignmentWithOffset() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingProcessingTimeWindows assigner =
+				SlidingProcessingTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(1000), Time.milliseconds(100));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(-3900, 1100),
+				timeWindow(-2900, 2100),
+				timeWindow(-1900, 3100),
+				timeWindow(-900, 4100),
+				timeWindow(100, 5100)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5099L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(100, 5100),
+				timeWindow(1100, 6100),
+				timeWindow(2100, 7100),
+				timeWindow(3100, 8100),
+				timeWindow(4100, 9100)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(1100, 6100),
+				timeWindow(2100, 7100),
+				timeWindow(3100, 8100),
+				timeWindow(4100, 9100),
+				timeWindow(5100, 10100)));
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		SlidingProcessingTimeWindows assigner = SlidingProcessingTimeWindows.of(Time.seconds(5), Time.seconds(1), Time.milliseconds(500));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(-4500, 500),
+				timeWindow(-3500, 1500),
+				timeWindow(-2500, 2500),
+				timeWindow(-1500, 3500),
+				timeWindow(-500, 4500)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5499L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(500, 5500),
+				timeWindow(1500, 6500),
+				timeWindow(2500, 7500),
+				timeWindow(3500, 8500),
+				timeWindow(4500, 9500)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), containsInAnyOrder(
+				timeWindow(500, 5500),
+				timeWindow(1500, 6500),
+				timeWindow(2500, 7500),
+				timeWindow(3500, 8500),
+				timeWindow(4500, 9500)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			SlidingProcessingTimeWindows.of(Time.seconds(-2), Time.seconds(1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+
+		try {
+			SlidingProcessingTimeWindows.of(Time.seconds(2), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+
+
+		try {
+			SlidingProcessingTimeWindows.of(Time.seconds(20), Time.seconds(10), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < slide and size > 0"));
+		}
+	}
+
+	@Test
+	public void testProperties() {
+		SlidingProcessingTimeWindows assigner = SlidingProcessingTimeWindows.of(Time.seconds(5), Time.milliseconds(100));
+
+		assertFalse(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(ProcessingTimeTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/StreamRecordMatchers.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/StreamRecordMatchers.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/StreamRecordMatchers.java
new file mode 100644
index 0000000..bb07996
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/StreamRecordMatchers.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
+
+/**
+ * Matchers that are useful for working with {@link StreamRecord StreamRecords}. This ...
+ */
+public class StreamRecordMatchers {
+
+  public static <T> Matcher<StreamRecord<? extends T>> isStreamRecord(
+          T value) {
+
+    return isStreamRecord(Matchers.equalTo(value));
+  }
+
+  public static <T> Matcher<StreamRecord<? extends T>> isStreamRecord(
+      T value,
+      long timestamp) {
+
+    return isStreamRecord(Matchers.equalTo(value), Matchers.equalTo(timestamp));
+  }
+
+  public static <T> Matcher<StreamRecord<? extends T>> isStreamRecord(
+          Matcher<? super T> valueMatcher) {
+    return new StreamRecordMatcher<>(valueMatcher, Matchers.anything());
+  }
+
+  public static <T> Matcher<StreamRecord<? extends T>> isStreamRecord(
+      Matcher<? super T> valueMatcher, Matcher<? super Long> timestampMatcher) {
+    return new StreamRecordMatcher<>(valueMatcher, timestampMatcher);
+  }
+
+  public static Matcher<TimeWindow> timeWindow(long start, long end) {
+    return Matchers.equalTo(new TimeWindow(start, end));
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @SafeVarargs
+  public static <W extends Window> Matcher<Iterable<W>> ofWindows(Matcher<W>... windows) {
+    return (Matcher) Matchers.containsInAnyOrder(windows);
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          T value) {
+    return isWindowedValue(Matchers.equalTo(value));
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          T value,
+          long timestamp) {
+    return isWindowedValue(Matchers.equalTo(value), Matchers.equalTo(timestamp));
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          T value,
+          long timestamp,
+          W window) {
+    return isWindowedValue(Matchers.equalTo(value), Matchers.equalTo(timestamp), Matchers.equalTo(window));
+  }
+
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher, long timestamp) {
+    return new WindowedValueMatcher<>(valueMatcher, Matchers.equalTo(timestamp), Matchers.anything());
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher, long timestamp, W window) {
+    return new WindowedValueMatcher<>(valueMatcher, Matchers.equalTo(timestamp), Matchers.equalTo(window));
+  }
+
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher) {
+    return new WindowedValueMatcher<>(valueMatcher, Matchers.anything(), Matchers.anything());
+  }
+
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher, Matcher<? super Long> timestampMatcher) {
+    return new WindowedValueMatcher<>(valueMatcher, timestampMatcher, Matchers.anything());
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher, long timestamp, Matcher<? super W> windowMatcher) {
+    return new WindowedValueMatcher<>(valueMatcher, Matchers.equalTo(timestamp), windowMatcher);
+  }
+
+  public static <T, W extends Window> Matcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> isWindowedValue(
+          Matcher<? super T> valueMatcher, Matcher<? super Long> timestampMatcher, Matcher<? super W> windowMatcher) {
+    return new WindowedValueMatcher<>(valueMatcher, timestampMatcher, windowMatcher);
+  }
+
+
+  private StreamRecordMatchers() {}
+
+  private static class StreamRecordMatcher<T> extends TypeSafeMatcher<StreamRecord<? extends T>> {
+
+    private Matcher<? super T> valueMatcher;
+    private Matcher<? super Long> timestampMatcher;
+
+    private StreamRecordMatcher(
+        Matcher<? super T> valueMatcher,
+        Matcher<? super Long> timestampMatcher) {
+      this.valueMatcher = valueMatcher;
+      this.timestampMatcher = timestampMatcher;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description
+          .appendText("a StreamRecordValue(").appendValue(valueMatcher)
+          .appendText(", ").appendValue(timestampMatcher)
+          .appendText(")");
+    }
+
+    @Override
+    protected boolean matchesSafely(StreamRecord<? extends T> streamRecord) {
+      return valueMatcher.matches(streamRecord.getValue())
+              && timestampMatcher.matches(streamRecord.getTimestamp());
+    }
+  }
+
+  private static class WindowedValueMatcher<T, W extends Window> extends TypeSafeMatcher<StreamRecord<? extends WindowedValue<? extends T, ? extends W>>> {
+
+    private Matcher<? super T> valueMatcher;
+    private Matcher<? super Long> timestampMatcher;
+    private Matcher<? super W> windowMatcher;
+
+
+    private WindowedValueMatcher(
+            Matcher<? super T> valueMatcher,
+            Matcher<? super Long> timestampMatcher,
+            Matcher<? super W> windowMatcher) {
+      this.valueMatcher = valueMatcher;
+      this.timestampMatcher = timestampMatcher;
+      this.windowMatcher = windowMatcher;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description
+              .appendText("a WindowedValue(").appendValue(valueMatcher)
+              .appendText(", ").appendValue(timestampMatcher)
+              .appendText(", ").appendValue(timestampMatcher)
+              .appendText(")");
+    }
+
+    @Override
+    protected boolean matchesSafely(StreamRecord<? extends WindowedValue<? extends T, ? extends W>> streamRecord) {
+      return valueMatcher.matches(streamRecord.getValue().value())
+              && timestampMatcher.matches(streamRecord.getTimestamp())
+              && windowMatcher.matches(streamRecord.getValue().window());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTestHarness.java
new file mode 100644
index 0000000..3fdc94f
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTestHarness.java
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.MergingState;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.TestInternalTimerService;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Utility for testing {@link Trigger} behaviour.
+ */
+public class TriggerTestHarness<T, W extends Window> {
+
+	private static final Integer KEY = 1;
+
+	private final Trigger<T, W> trigger;
+	private final TypeSerializer<W> windowSerializer;
+
+	private final HeapKeyedStateBackend<Integer> stateBackend;
+	private final TestInternalTimerService<Integer, W> internalTimerService;
+
+	public TriggerTestHarness(
+			Trigger<T, W> trigger,
+			TypeSerializer<W> windowSerializer) throws Exception {
+		this.trigger = trigger;
+		this.windowSerializer = windowSerializer;
+
+		// we only ever use one key, other tests make sure that windows work across different
+		// keys
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		MemoryStateBackend backend = new MemoryStateBackend();
+
+		@SuppressWarnings("unchecked")
+		HeapKeyedStateBackend<Integer> stateBackend = (HeapKeyedStateBackend<Integer>) backend.createKeyedStateBackend(dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				1,
+				new KeyGroupRange(0, 0),
+				new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
+		this.stateBackend = stateBackend;
+
+		this.stateBackend.setCurrentKey(0);
+
+		this.internalTimerService = new TestInternalTimerService<>(new KeyContext() {
+			@Override
+			public void setCurrentKey(Object key) {
+				// ignore
+			}
+
+			@Override
+			public Object getCurrentKey() {
+				return KEY;
+			}
+		});
+	}
+
+	public int numProcessingTimeTimers() {
+		return internalTimerService.numProcessingTimeTimers();
+	}
+
+	public int numProcessingTimeTimers(W window) {
+		return internalTimerService.numProcessingTimeTimers(window);
+	}
+
+	public int numEventTimeTimers() {
+		return internalTimerService.numEventTimeTimers();
+	}
+
+	public int numEventTimeTimers(W window) {
+		return internalTimerService.numEventTimeTimers(window);
+	}
+
+	public int numStateEntries() {
+		return stateBackend.numStateEntries();
+	}
+
+	public int numStateEntries(W window) {
+		return stateBackend.numStateEntries(window);
+	}
+
+	/**
+	 * Injects one element into the trigger for the given window and returns the result of
+	 * {@link Trigger#onElement(Object, long, Window, Trigger.TriggerContext)}
+	 */
+	public TriggerResult processElement(StreamRecord<T> element, W window) throws Exception {
+		TestTriggerContext<Integer, W> triggerContext = new TestTriggerContext<>(
+				KEY,
+				window,
+				internalTimerService,
+				stateBackend,
+				windowSerializer);
+		return trigger.onElement(element.getValue(), element.getTimestamp(), window, triggerContext);
+	}
+
+	/**
+	 * Advanced processing time and checks whether we have exactly one firing for the given
+	 * window. The result of {@link Trigger#onProcessingTime(long, Window, Trigger.TriggerContext)}
+	 * is returned for that firing.
+	 */
+	public TriggerResult advanceProcessingTime(long time, W window) throws Exception {
+		Collection<Tuple2<W, TriggerResult>> firings = advanceProcessingTime(time);
+
+		if (firings.size() != 1) {
+			throw new IllegalStateException("Must have exactly one timer firing. Fired timers: " + firings);
+		}
+
+		Tuple2<W, TriggerResult> firing = firings.iterator().next();
+
+		if (!firing.f0.equals(window)) {
+			throw new IllegalStateException("Trigger fired for another window.");
+		}
+
+		return firing.f1;
+	}
+
+	/**
+	 * Advanced the watermark and checks whether we have exactly one firing for the given
+	 * window. The result of {@link Trigger#onEventTime(long, Window, Trigger.TriggerContext)}
+	 * is returned for that firing.
+	 */
+	public TriggerResult advanceWatermark(long time, W window) throws Exception {
+		Collection<Tuple2<W, TriggerResult>> firings = advanceWatermark(time);
+
+		if (firings.size() != 1) {
+			throw new IllegalStateException("Must have exactly one timer firing. Fired timers: " + firings);
+		}
+
+		Tuple2<W, TriggerResult> firing = firings.iterator().next();
+
+		if (!firing.f0.equals(window)) {
+			throw new IllegalStateException("Trigger fired for another window.");
+		}
+
+		return firing.f1;
+	}
+
+	/**
+	 * Advanced processing time and processes any timers that fire because of this. The
+	 * window and {@link TriggerResult} for each firing are returned.
+	 */
+	public Collection<Tuple2<W, TriggerResult>> advanceProcessingTime(long time) throws Exception {
+		Collection<TestInternalTimerService.Timer<Integer, W>> firedTimers =
+				internalTimerService.advanceProcessingTime(time);
+
+		Collection<Tuple2<W, TriggerResult>> result = new ArrayList<>();
+
+		for (TestInternalTimerService.Timer<Integer, W> timer : firedTimers) {
+			TestTriggerContext<Integer, W> triggerContext = new TestTriggerContext<>(
+					KEY,
+					timer.getNamespace(),
+					internalTimerService,
+					stateBackend,
+					windowSerializer);
+
+			TriggerResult triggerResult =
+					trigger.onProcessingTime(timer.getTimestamp(), timer.getNamespace(), triggerContext);
+
+			result.add(new Tuple2<>(timer.getNamespace(), triggerResult));
+		}
+
+		return result;
+	}
+
+	/**
+	 * Advanced the watermark and processes any timers that fire because of this. The
+	 * window and {@link TriggerResult} for each firing are returned.
+	 */
+	public Collection<Tuple2<W, TriggerResult>> advanceWatermark(long time) throws Exception {
+		Collection<TestInternalTimerService.Timer<Integer, W>> firedTimers =
+				internalTimerService.advanceWatermark(time);
+
+		Collection<Tuple2<W, TriggerResult>> result = new ArrayList<>();
+
+		for (TestInternalTimerService.Timer<Integer, W> timer : firedTimers) {
+			TestTriggerContext<Integer, W> triggerContext = new TestTriggerContext<>(
+					KEY,
+					timer.getNamespace(),
+					internalTimerService,
+					stateBackend,
+					windowSerializer);
+
+			TriggerResult triggerResult =
+					trigger.onEventTime(timer.getTimestamp(), timer.getNamespace(), triggerContext);
+
+			result.add(new Tuple2<>(timer.getNamespace(), triggerResult));
+		}
+
+		return result;
+	}
+
+	/**
+	 * Calls {@link Trigger#onMerge(Window, Trigger.OnMergeContext)} with the given
+	 * parameters. This also calls {@link Trigger#clear(Window, Trigger.TriggerContext)} on the
+	 * merged windows as does {@link WindowOperator}.
+	 */
+	public void mergeWindows(W targetWindow, Collection<W> mergedWindows) throws Exception {
+		TestOnMergeContext<Integer, W> onMergeContext = new TestOnMergeContext<>(
+				KEY,
+				targetWindow,
+				mergedWindows,
+				internalTimerService,
+				stateBackend,
+				windowSerializer);
+		trigger.onMerge(targetWindow, onMergeContext);
+
+		for (W mergedWindow : mergedWindows) {
+			clearTriggerState(mergedWindow);
+		}
+	}
+
+	/**
+	 * Calls {@link Trigger#clear(Window, Trigger.TriggerContext)} for the given window.
+	 */
+	public void clearTriggerState(W window) throws Exception {
+		TestTriggerContext<Integer, W> triggerContext = new TestTriggerContext<>(
+				KEY,
+				window,
+				internalTimerService,
+				stateBackend,
+				windowSerializer);
+		trigger.clear(window, triggerContext);
+	}
+
+	private static class TestTriggerContext<K, W extends Window> implements Trigger.TriggerContext {
+
+		protected final InternalTimerService<W> timerService;
+		protected final KeyedStateBackend<Integer> stateBackend;
+		protected final K key;
+		protected final W window;
+		protected final TypeSerializer<W> windowSerializer;
+
+		TestTriggerContext(
+				K key,
+				W window,
+				InternalTimerService<W> timerService,
+				KeyedStateBackend<Integer> stateBackend,
+				TypeSerializer<W> windowSerializer) {
+			this.key = key;
+			this.window = window;
+			this.timerService = timerService;
+			this.stateBackend = stateBackend;
+			this.windowSerializer = windowSerializer;
+		}
+
+		@Override
+		public long getCurrentProcessingTime() {
+			return timerService.currentProcessingTime();
+		}
+
+		@Override
+		public MetricGroup getMetricGroup() {
+			return null;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return timerService.currentWatermark();
+		}
+
+		@Override
+		public void registerProcessingTimeTimer(long time) {
+			timerService.registerProcessingTimeTimer(window, time);
+		}
+
+		@Override
+		public void registerEventTimeTimer(long time) {
+			timerService.registerEventTimeTimer(window, time);
+		}
+
+		@Override
+		public void deleteProcessingTimeTimer(long time) {
+			timerService.deleteProcessingTimeTimer(window, time);
+		}
+
+		@Override
+		public void deleteEventTimeTimer(long time) {
+			timerService.deleteEventTimeTimer(window, time);
+		}
+
+		@Override
+		public <S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor) {
+			try {
+				return stateBackend.getPartitionedState(window, windowSerializer, stateDescriptor);
+			} catch (Exception e) {
+				throw new RuntimeException("Error getting state", e);
+			}
+		}
+
+		@Override
+		public <S extends Serializable> ValueState<S> getKeyValueState(
+				String name, Class<S> stateType, S defaultState) {
+			return getPartitionedState(new ValueStateDescriptor<>(name, stateType, defaultState));
+		}
+
+		@Override
+		public <S extends Serializable> ValueState<S> getKeyValueState(
+				String name, TypeInformation<S> stateType, S defaultState) {
+			return getPartitionedState(new ValueStateDescriptor<>(name, stateType, defaultState));
+		}
+	}
+
+	private static class TestOnMergeContext<K, W extends Window> extends TestTriggerContext<K, W> implements Trigger.OnMergeContext {
+
+		private final Collection<W> mergedWindows;
+
+		public TestOnMergeContext(
+				K key,
+				W targetWindow,
+				Collection<W> mergedWindows,
+				InternalTimerService<W> timerService,
+				KeyedStateBackend<Integer> stateBackend,
+				TypeSerializer<W> windowSerializer) {
+			super(key, targetWindow, timerService, stateBackend, windowSerializer);
+
+			this.mergedWindows = mergedWindows;
+		}
+
+		@Override
+		public <S extends MergingState<?, ?>> void mergePartitionedState(StateDescriptor<S, ?> stateDescriptor) {
+			try {
+				stateBackend.mergePartitionedStates(window, mergedWindows, windowSerializer, stateDescriptor);
+			} catch (Exception e) {
+				throw new RuntimeException("Error merging state", e);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingEventTimeWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingEventTimeWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingEventTimeWindowsTest.java
new file mode 100644
index 0000000..2373a86
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingEventTimeWindowsTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for {@link TumblingEventTimeWindows}
+ */
+public class TumblingEventTimeWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.milliseconds(5000));
+
+		assertThat(assigner.assignWindows("String", 0L, mockContext), contains(timeWindow(0, 5000)));
+		assertThat(assigner.assignWindows("String", 4999L, mockContext), contains(timeWindow(0, 5000)));
+		assertThat(assigner.assignWindows("String", 5000L, mockContext), contains(timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testWindowAssignmentWithOffset() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(100));
+
+		assertThat(assigner.assignWindows("String", 100L, mockContext), contains(timeWindow(100, 5100)));
+		assertThat(assigner.assignWindows("String", 5099L, mockContext), contains(timeWindow(100, 5100)));
+		assertThat(assigner.assignWindows("String", 5100L, mockContext), contains(timeWindow(5100, 10100)));
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.seconds(5), Time.seconds(1));
+
+		assertThat(assigner.assignWindows("String", 1000L, mockContext), contains(timeWindow(1000, 6000)));
+		assertThat(assigner.assignWindows("String", 5999L, mockContext), contains(timeWindow(1000, 6000)));
+		assertThat(assigner.assignWindows("String", 6000L, mockContext), contains(timeWindow(6000, 11000)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			TumblingEventTimeWindows.of(Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+
+		try {
+			TumblingEventTimeWindows.of(Time.seconds(10), Time.seconds(20));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+
+		try {
+			TumblingEventTimeWindows.of(Time.seconds(10), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+	}
+
+	@Test
+	public void testProperties() {
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.seconds(5), Time.milliseconds(100));
+
+		assertTrue(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(EventTimeTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingProcessingTimeWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingProcessingTimeWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingProcessingTimeWindowsTest.java
new file mode 100644
index 0000000..348b6fa
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TumblingProcessingTimeWindowsTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link TumblingProcessingTimeWindows}
+ */
+public class TumblingProcessingTimeWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingProcessingTimeWindows assigner = TumblingProcessingTimeWindows.of(Time.milliseconds(5000));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(0L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(0, 5000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(4999L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(0, 5000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testWindowAssignmentWithOffset() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingProcessingTimeWindows assigner = TumblingProcessingTimeWindows.of(Time.milliseconds(5000), Time.milliseconds(100));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(100, 5100)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5099L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(100, 5100)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5100L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(5100, 10100)));
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		TumblingProcessingTimeWindows assigner = TumblingProcessingTimeWindows.of(Time.seconds(5), Time.seconds(1));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(1000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(1000, 6000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5999L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(1000, 6000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(6000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(6000, 11000)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			TumblingProcessingTimeWindows.of(Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+
+		try {
+			TumblingProcessingTimeWindows.of(Time.seconds(10), Time.seconds(20));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+
+		try {
+			TumblingProcessingTimeWindows.of(Time.seconds(10), Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 <= offset < size"));
+		}
+	}
+
+	@Test
+	public void testProperties() {
+		TumblingProcessingTimeWindows assigner = TumblingProcessingTimeWindows.of(Time.seconds(5), Time.milliseconds(100));
+
+		assertFalse(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(ProcessingTimeTrigger.class));
+	}
+}


[07/10] flink git commit: [hotfix] Make MergingWindowSet resilient to misbehaving window assigners

Posted by al...@apache.org.
[hotfix] Make MergingWindowSet resilient to misbehaving window assigners


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

Branch: refs/heads/release-1.2
Commit: e3bab4afc3185bcfd4e7478350bf58572fe20567
Parents: f4cd7d0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Nov 4 10:17:55 2016 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../operators/windowing/MergingWindowSet.java   |   4 +-
 .../windowing/MergingWindowSetTest.java         | 108 +++++++++++++++++++
 2 files changed, 111 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e3bab4af/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
index 06cacad..b79a3fa 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
@@ -171,6 +171,7 @@ public class MergingWindowSet<W extends Window> {
 				});
 
 		W resultWindow = newWindow;
+		boolean mergedNewWindow = false;
 
 		// perform the merge
 		for (Map.Entry<W, Collection<W>> c: mergeResults.entrySet()) {
@@ -180,6 +181,7 @@ public class MergingWindowSet<W extends Window> {
 			// if our new window is in the merged windows make the merge result the
 			// result window
 			if (mergedWindows.remove(newWindow)) {
+				mergedNewWindow = true;
 				resultWindow = mergeResult;
 			}
 
@@ -213,7 +215,7 @@ public class MergingWindowSet<W extends Window> {
 		}
 
 		// the new window created a new, self-contained window without merging
-		if (resultWindow.equals(newWindow) && mergeResults.isEmpty()) {
+		if (mergeResults.isEmpty() || (resultWindow.equals(newWindow) && !mergedNewWindow)) {
 			this.mapping.put(resultWindow, resultWindow);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/e3bab4af/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
index 7c1fa93..46169a8 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
@@ -18,15 +18,25 @@
 package org.apache.flink.streaming.runtime.operators.windowing;
 
 import com.google.common.collect.Lists;
+import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
 import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.junit.Test;
 import org.mockito.Matchers;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 
 import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.hasItem;
@@ -44,6 +54,40 @@ import static org.mockito.Mockito.*;
  */
 public class MergingWindowSetTest {
 
+	/**
+	 * This test uses a special (misbehaving) {@code MergingWindowAssigner} that produces cases
+	 * where windows that don't overlap with the newly added window are being merged. We verify
+	 * that the merging window set is nevertheless correct and contains all added windows.
+	 */
+	@Test
+	public void testNonEagerMerging() throws Exception {
+		@SuppressWarnings("unchecked")
+		ListState<Tuple2<TimeWindow, TimeWindow>> mockState = mock(ListState.class);
+
+		MergingWindowSet<TimeWindow> windowSet =
+				new MergingWindowSet<>(new NonEagerlyMergingWindowAssigner(3000), mockState);
+
+		TestingMergeFunction mergeFunction = new TestingMergeFunction();
+
+		TimeWindow result;
+
+		mergeFunction.reset();
+		result = windowSet.addWindow(new TimeWindow(0, 2), mergeFunction);
+		assertNotNull(windowSet.getStateWindow(result));
+
+		mergeFunction.reset();
+		result = windowSet.addWindow(new TimeWindow(2, 5), mergeFunction);
+		assertNotNull(windowSet.getStateWindow(result));
+
+		mergeFunction.reset();
+		result = windowSet.addWindow(new TimeWindow(1, 2), mergeFunction);
+		assertNotNull(windowSet.getStateWindow(result));
+
+		mergeFunction.reset();
+		result = windowSet.addWindow(new TimeWindow(10, 12), mergeFunction);
+		assertNotNull(windowSet.getStateWindow(result));
+	}
+
 	@Test
 	public void testIncrementalMerging() throws Exception {
 		@SuppressWarnings("unchecked")
@@ -392,4 +436,68 @@ public class MergingWindowSetTest {
 			this.sources = mergedWindows;
 		}
 	}
+
+	/**
+	 * A special {@link MergingWindowAssigner} that let's windows get larger which leads to windows
+	 * being merged lazily.
+	 */
+	static class NonEagerlyMergingWindowAssigner extends MergingWindowAssigner<Object, TimeWindow> {
+		private static final long serialVersionUID = 1L;
+
+		protected long sessionTimeout;
+
+		public NonEagerlyMergingWindowAssigner(long sessionTimeout) {
+			this.sessionTimeout = sessionTimeout;
+		}
+
+		@Override
+		public Collection<TimeWindow> assignWindows(Object element, long timestamp, WindowAssignerContext context) {
+			return Collections.singletonList(new TimeWindow(timestamp, timestamp + sessionTimeout));
+		}
+
+		@Override
+		public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
+			return EventTimeTrigger.create();
+		}
+
+		@Override
+		public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig) {
+			return new TimeWindow.Serializer();
+		}
+
+		@Override
+		public boolean isEventTime() {
+			return true;
+		}
+
+		/**
+		 * Merge overlapping {@link TimeWindow}s.
+		 */
+		public void mergeWindows(Collection<TimeWindow> windows, MergingWindowAssigner.MergeCallback<TimeWindow> c) {
+
+			TimeWindow earliestStart = null;
+
+			for (TimeWindow win : windows) {
+				if (earliestStart == null) {
+					earliestStart = win;
+				} else if (win.getStart() < earliestStart.getStart()) {
+					earliestStart = win;
+				}
+			}
+
+			List<TimeWindow> associatedWindows = new ArrayList<>();
+
+			for (TimeWindow win : windows) {
+				if (win.getStart() < earliestStart.getEnd() && win.getStart() >= earliestStart.getStart()) {
+					associatedWindows.add(win);
+				}
+			}
+
+			TimeWindow target = new TimeWindow(earliestStart.getStart(), earliestStart.getEnd() + 1);
+
+			if (associatedWindows.size() > 1) {
+				c.merge(associatedWindows, target);
+			}
+		}
+	}
 }


[09/10] flink git commit: [FLINK-4994] Don't Clear Trigger State and Merging Window Set When Purging

Posted by al...@apache.org.
[FLINK-4994] Don't Clear Trigger State and Merging Window Set When Purging

Before, when a Trigger returns TriggerResult.PURGE from any of the
on*() methods the WindowOperator will clear all state of that window
(window contents, merging window set) and call Trigger.clear() so that the
Trigger can clean up its state/timers.

This was problematic in some cases. For example, with merging windows (session
windows) this means that a late-arriving element will not be put into the
session that was previously built up but will be put into a completely new
session that only contains this one element.

The new behaviour is this:
 * Only clean window contents on PURGE
 * Register cleanup timer for any window, don't delete this on PURGE
 * When the cleanup timer fires: clean window state, clean merging window set,
call Trigger.clear() to allow it to clean state/timers


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

Branch: refs/heads/release-1.2
Commit: a9189c2055fd3c10d741f63208aadca7fb4218f5
Parents: 704b411
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Nov 2 11:51:07 2016 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../windowing/EvictingWindowOperator.java       | 118 ++++++++------
 .../operators/windowing/WindowOperator.java     | 155 +++++++++++--------
 .../operators/windowing/WindowOperatorTest.java |  11 +-
 3 files changed, 162 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a9189c20/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 8c73878..17b3984 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
@@ -154,14 +154,13 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 						// if we have no state, there is nothing to do
 						continue;
 					}
-					fire(actualWindow, contents, windowState);
+					emitWindowContents(actualWindow, contents, windowState);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(actualWindow, windowState, mergingWindows);
-				} else {
-					registerCleanupTimer(actualWindow);
+					windowState.clear();
 				}
+				registerCleanupTimer(actualWindow);
 			}
 
 			mergingWindows.persist();
@@ -190,14 +189,13 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 						// if we have no state, there is nothing to do
 						continue;
 					}
-					fire(window, contents, windowState);
+					emitWindowContents(window, contents, windowState);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(window, windowState, null);
-				} else {
-					registerCleanupTimer(window);
+					windowState.clear();
 				}
+				registerCleanupTimer(window);
 			}
 		}
 	}
@@ -217,12 +215,14 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 			mergingWindows = getMergingWindowSet();
 			W stateWindow = mergingWindows.getStateWindow(context.window);
 			if (stateWindow == null) {
-				// then the window is already purged and this is a cleanup
-				// timer set due to allowed lateness that has nothing to clean,
-				// so it is safe to just ignore
-				return;
+				// timer firing for non-existent window, still have to run the cleanup logic
+				windowState = null;
+			} else {
+				windowState = getPartitionedState(
+						stateWindow,
+						windowSerializer,
+						windowStateDescriptor);
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
 		} else {
 			windowState = getPartitionedState(
 					context.window,
@@ -230,19 +230,28 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 					windowStateDescriptor);
 		}
 
-		Iterable<StreamRecord<IN>> contents = windowState.get();
-		if (contents == null) {
-			// if we have no state, there is nothing to do
-			return;
+		Iterable<StreamRecord<IN>> contents = null;
+		if (windowState != null) {
+			contents = windowState.get();
 		}
 
-		TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
-		if (triggerResult.isFire()) {
-			fire(context.window, contents, windowState);
+		if (contents != null) {
+			TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
+			if (triggerResult.isFire()) {
+				emitWindowContents(context.window, contents, windowState);
+			}
+			if (triggerResult.isPurge()) {
+				windowState.clear();
+			}
+		}
+
+		if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
+			clearAllState(context.window, windowState, mergingWindows);
 		}
 
-		if (triggerResult.isPurge() || (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+		if (mergingWindows != null) {
+			// need to make sure to update the merging state in state
+			mergingWindows.persist();
 		}
 	}
 
@@ -260,33 +269,44 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 			mergingWindows = getMergingWindowSet();
 			W stateWindow = mergingWindows.getStateWindow(context.window);
 			if (stateWindow == null) {
-				// then the window is already purged and this is a cleanup
-				// timer set due to allowed lateness that has nothing to clean,
-				// so it is safe to just ignore
-				return;
+				// timer firing for non-existent window, still have to run the cleanup logic
+				windowState = null;
+			} else {
+				windowState = getPartitionedState(
+						stateWindow,
+						windowSerializer,
+						windowStateDescriptor);
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
 		} else {
 			windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
 		}
 
-		Iterable<StreamRecord<IN>> contents = windowState.get();
-		if (contents == null) {
-			// if we have no state, there is nothing to do
-			return;
+		Iterable<StreamRecord<IN>> contents = null;
+		if (windowState != null) {
+			contents = windowState.get();
 		}
 
-		TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
-		if (triggerResult.isFire()) {
-			fire(context.window, contents, windowState);
+		if (contents != null) {
+			TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
+			if (triggerResult.isFire()) {
+				emitWindowContents(context.window, contents, windowState);
+			}
+			if (triggerResult.isPurge()) {
+				windowState.clear();
+			}
 		}
 
-		if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+		if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
+			clearAllState(context.window, windowState, mergingWindows);
+		}
+
+		if (mergingWindows != null) {
+			// need to make sure to update the merging state in state
+			mergingWindows.persist();
 		}
 	}
 
-	private void fire(W window, Iterable<StreamRecord<IN>> contents, ListState<StreamRecord<IN>> windowState) throws Exception {
+	private void emitWindowContents(W window, Iterable<StreamRecord<IN>> contents, ListState<StreamRecord<IN>> windowState) throws Exception {
 		timestampedCollector.setAbsoluteTimestamp(window.maxTimestamp());
 
 		// Work around type system restrictions...
@@ -320,6 +340,18 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 		}
 	}
 
+	private void clearAllState(
+			W window,
+			ListState<StreamRecord<IN>> windowState,
+			MergingWindowSet<W> mergingWindows) throws Exception {
+
+		windowState.clear();
+		context.clear();
+		if (mergingWindows != null) {
+			mergingWindows.retireWindow(window);
+			mergingWindows.persist();
+		}
+	}
 
 	/**
 	 * {@code EvictorContext} is a utility for handling {@code Evictor} invocations. It can be reused
@@ -366,18 +398,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 		}
 	}
 
-	private void cleanup(W window,
-						ListState<StreamRecord<IN>> windowState,
-						MergingWindowSet<W> mergingWindows) throws Exception {
-
-		windowState.clear();
-		if (mergingWindows != null) {
-			mergingWindows.retireWindow(window);
-			mergingWindows.persist();
-		}
-		context.clear();
-	}
-
 	@Override
 	public void open() throws Exception {
 		super.open();

http://git-wip-us.apache.org/repos/asf/flink/blob/a9189c20/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 5ed5a4e..3144b6d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -348,14 +348,13 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					if (contents == null) {
 						continue;
 					}
-					fire(actualWindow, contents);
+					emitWindowContents(actualWindow, contents);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(actualWindow, windowState, mergingWindows);
-				} else {
-					registerCleanupTimer(actualWindow);
+					windowState.clear();
 				}
+				registerCleanupTimer(actualWindow);
 			}
 
 			// need to make sure to update the merging state in state
@@ -382,14 +381,13 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					if (contents == null) {
 						continue;
 					}
-					fire(window, contents);
+					emitWindowContents(window, contents);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(window, windowState, null);
-				} else {
-					registerCleanupTimer(window);
+					windowState.clear();
 				}
+				registerCleanupTimer(window);
 			}
 		}
 	}
@@ -406,32 +404,40 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			mergingWindows = getMergingWindowSet();
 			W stateWindow = mergingWindows.getStateWindow(context.window);
 			if (stateWindow == null) {
-				// then the window is already purged and this is a cleanup
-				// timer set due to allowed lateness that has nothing to clean,
-				// so it is safe to just ignore
-				return;
+				// timer firing for non-existent window, ignore
+				windowState = null;
+			} else {
+				windowState = getPartitionedState(
+						stateWindow,
+						windowSerializer,
+						windowStateDescriptor);
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
 		} else {
-			windowState = getPartitionedState(
-					context.window,
-					windowSerializer,
-					windowStateDescriptor);
+			windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
 		}
 
-		ACC contents = windowState.get();
-		if (contents == null) {
-			// if we have no state, there is nothing to do
-			return;
+		ACC contents = null;
+		if (windowState != null) {
+			contents = windowState.get();
 		}
 
-		TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
-		if (triggerResult.isFire()) {
-			fire(context.window, contents);
+		if (contents != null) {
+			TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
+			if (triggerResult.isFire()) {
+				emitWindowContents(context.window, contents);
+			}
+			if (triggerResult.isPurge()) {
+				windowState.clear();
+			}
 		}
 
-		if (triggerResult.isPurge() || (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+		if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
+			clearAllState(context.window, windowState, mergingWindows);
+		}
+
+		if (mergingWindows != null) {
+			// need to make sure to update the merging state in state
+			mergingWindows.persist();
 		}
 	}
 
@@ -447,54 +453,67 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			mergingWindows = getMergingWindowSet();
 			W stateWindow = mergingWindows.getStateWindow(context.window);
 			if (stateWindow == null) {
-				// then the window is already purged and this is a cleanup
-				// timer set due to allowed lateness that has nothing to clean,
-				// so it is safe to just ignore
-				return;
+				// timer firing for non-existent window, ignore
+				windowState = null;
+			} else {
+				windowState = getPartitionedState(
+						stateWindow,
+						windowSerializer,
+						windowStateDescriptor);
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
 		} else {
 			windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
 		}
 
-		ACC contents = windowState.get();
-		if (contents == null) {
-			// if we have no state, there is nothing to do
-			return;
+		ACC contents = null;
+		if (windowState != null) {
+			contents = windowState.get();
 		}
 
-		TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
-		if (triggerResult.isFire()) {
-			fire(context.window, contents);
+		if (contents != null) {
+			TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
+			if (triggerResult.isFire()) {
+				emitWindowContents(context.window, contents);
+			}
+			if (triggerResult.isPurge()) {
+				windowState.clear();
+			}
 		}
 
-		if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+		if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
+			clearAllState(context.window, windowState, mergingWindows);
+		}
+
+		if (mergingWindows != null) {
+			// need to make sure to update the merging state in state
+			mergingWindows.persist();
 		}
 	}
 
 	/**
-	 * Cleans up the window state if the provided {@link TriggerResult} requires so, or if it
-	 * is time to do so (see {@link #isCleanupTime(Window, long)}). The caller must ensure that the
+	 * Drops all state for the given window and calls
+	 * {@link Trigger#clear(Window, Trigger.TriggerContext)}.
+	 *
+	 * <p>The caller must ensure that the
 	 * correct key is set in the state backend and the context object.
 	 */
-	private void cleanup(W window,
-						AppendingState<IN, ACC> windowState,
-						MergingWindowSet<W> mergingWindows) throws Exception {
+	private void clearAllState(
+			W window,
+			AppendingState<IN, ACC> windowState,
+			MergingWindowSet<W> mergingWindows) throws Exception {
 		windowState.clear();
+		context.clear();
 		if (mergingWindows != null) {
 			mergingWindows.retireWindow(window);
 			mergingWindows.persist();
 		}
-		context.clear();
 	}
 
 	/**
-	 * Triggers the window computation if the provided {@link TriggerResult} requires so.
-	 * The caller must ensure that the correct key is set in the state backend and the context object.
+	 * Emits the contents of the given window using the {@link InternalWindowFunction}.
 	 */
 	@SuppressWarnings("unchecked")
-	private void fire(W window, ACC contents) throws Exception {
+	private void emitWindowContents(W window, ACC contents) throws Exception {
 		timestampedCollector.setAbsoluteTimestamp(window.maxTimestamp());
 		userFunction.apply(context.key, context.window, contents, timestampedCollector);
 	}
@@ -517,12 +536,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	}
 
 	/**
-	 * Decides if a window is currently late or not, based on the current
-	 * watermark, i.e. the current event time, and the allowed lateness.
-	 * @param window
-	 * 					The collection of windows returned by the {@link WindowAssigner}.
-	 * @return The windows (among the {@code eligibleWindows}) for which the element should still be
-	 * 					considered when triggering.
+	 * Returns {@code true} if the watermark is after the end timestamp plus the allowed lateness
+	 * of the given window.
 	 */
 	protected boolean isLate(W window) {
 		return (windowAssigner.isEventTime() && (cleanupTime(window) <= internalTimerService.currentWatermark()));
@@ -535,6 +550,11 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 */
 	protected void registerCleanupTimer(W window) {
 		long cleanupTime = cleanupTime(window);
+		if (cleanupTime == Long.MAX_VALUE) {
+			// don't set a GC timer for "end of time"
+			return;
+		}
+
 		if (windowAssigner.isEventTime()) {
 			context.registerEventTimeTimer(cleanupTime);
 		} else {
@@ -549,6 +569,10 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 */
 	protected void deleteCleanupTimer(W window) {
 		long cleanupTime = cleanupTime(window);
+		if (cleanupTime == Long.MAX_VALUE) {
+			// no need to clean up because we didn't set one
+			return;
+		}
 		if (windowAssigner.isEventTime()) {
 			context.deleteEventTimeTimer(cleanupTime);
 		} else {
@@ -566,24 +590,19 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 * @param window the window whose cleanup time we are computing.
 	 */
 	private long cleanupTime(W window) {
-		long cleanupTime = window.maxTimestamp() + allowedLateness;
-		return cleanupTime >= window.maxTimestamp() ? cleanupTime : Long.MAX_VALUE;
+		if (windowAssigner.isEventTime()) {
+			long cleanupTime = window.maxTimestamp() + allowedLateness;
+			return cleanupTime >= window.maxTimestamp() ? cleanupTime : Long.MAX_VALUE;
+		} else {
+			return window.maxTimestamp();
+		}
 	}
 
 	/**
-	 * Decides if it is time to clean up the window state.
-	 * Clean up time for a window is:
-	 * 		<li> if it is event time, after the watermark passes the end of the window plus the user-specified allowed lateness
-	 * 		<li> if it is processing time, after the processing time at the node passes the end of the window.
-	 * 	@param window
-	 * 					the window to clean
-	 *  @param time
-	 *  				the current time (event or processing depending on the {@link WindowAssigner}
-	 *  @return {@code true} if it is time to clean up the window state, {@code false} otherwise.
+	 * Returns {@code true} if the given time is the cleanup time for the given window.
 	 */
 	protected final boolean isCleanupTime(W window, long time) {
-		long cleanupTime = cleanupTime(window);
-		return  cleanupTime == time;
+		return time == cleanupTime(window);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/a9189c20/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index 0e2d1e8..e682e2d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -1599,20 +1599,21 @@ public class WindowOperatorTest extends TestLogger {
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599));
 		expected.add(new Watermark(14600));
 
-		// dropped as late
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000));
 
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 14600L), 14599));
+
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
 		testHarness.processWatermark(new Watermark(20000));
 
-		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 14500L, 17500L), 17499));
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 17500L), 17499));
 		expected.add(new Watermark(20000));
 
 		testHarness.processWatermark(new Watermark(100000));
 		expected.add(new Watermark(100000));
 
 		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple2ResultSortComparator());
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
 		testHarness.close();
 	}
 
@@ -1778,7 +1779,7 @@ public class WindowOperatorTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000));
 
-		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 13000L), 12999));
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 1000L, 14600L), 14599));
 
 		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
@@ -1786,7 +1787,7 @@ public class WindowOperatorTest extends TestLogger {
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
 		testHarness.processWatermark(new Watermark(20000));
 
-		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 14500L, 17500L), 17499));
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 1000L, 17500L), 17499));
 		expected.add(new Watermark(20000));
 
 		testHarness.processWatermark(new Watermark(100000));


[10/10] flink git commit: [hotfix] Use correct ClassLoader in ContinuousFileProcessingMigrationTest

Posted by al...@apache.org.
[hotfix] Use correct ClassLoader in ContinuousFileProcessingMigrationTest


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

Branch: refs/heads/release-1.2
Commit: 1d362f514d105845035f43a66fa7b994c3fdd57d
Parents: 16b63c2
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Jan 13 14:16:51 2017 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../flink/hdfstests/ContinuousFileProcessingMigrationTest.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1d362f51/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
index 0915005..440bfcc 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
@@ -35,7 +35,6 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
 import org.apache.flink.streaming.api.operators.StreamSource;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorTest;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
@@ -107,7 +106,7 @@ public class ContinuousFileProcessingMigrationTest {
 	}
 
 	private static String getResourceFilename(String filename) {
-		ClassLoader cl = WindowOperatorTest.class.getClassLoader();
+		ClassLoader cl = ContinuousFileProcessingMigrationTest.class.getClassLoader();
 		URL resource = cl.getResource(filename);
 		return resource.getFile();
 	}


[04/10] flink git commit: [FLINK-4552] Refactor WindowOperator/Trigger Tests

Posted by al...@apache.org.
[FLINK-4552] Refactor WindowOperator/Trigger Tests

Before, tests for WindowOperator, WindowAssigner, Trigger and
WindowFunction were all conflated in WindowOperatorTest. All of these
test that a certain combination of a Trigger, WindowAssigner and
WindowFunction produce the expected output.

This change modularizes these tests and spreads them out across multiple
files. For example, one per trigger/window assigner.

The new WindowOperatorContractTest verifies that the interaction between
WindowOperator and the various other parts works as expected, that the
correct methods on Trigger and WindowFunction are called at the expected
time and that snapshotting, timers, cleanup etc. work correctly. These
tests also verify that the different state types and WindowFunctions
work correctly.

For trigger tests this introduces TriggerTestHarness. This can be used
to inject elements into Triggers they fire at the correct times. The
actual output of the WindowFunction is not important for these tests.
The new tests also make sure that triggers correctly clean up state and
timers.

WindowAssigner tests verify the behaviour of window assigners in
isolation.  They also test, for example, whether offset parameter of
time-based windows work correctly.

We keep the old WindowOperatorTest because it still provides some level
of coverage and doesn't take long to run.


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

Branch: refs/heads/release-1.2
Commit: bb8586e50be9e7d0b1fba5569579def595c53217
Parents: a9189c2
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Sep 5 12:01:11 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../state/heap/HeapKeyedStateBackend.java       |   45 +
 .../runtime/state/MemoryStateBackendTest.java   |   43 +
 .../assigners/EventTimeSessionWindows.java      |    4 +
 .../api/windowing/assigners/GlobalWindows.java  |    4 +-
 .../assigners/ProcessingTimeSessionWindows.java |    4 +
 .../assigners/SlidingEventTimeWindows.java      |   28 +-
 .../assigners/SlidingProcessingTimeWindows.java |   30 +-
 .../assigners/TumblingEventTimeWindows.java     |   36 +-
 .../TumblingProcessingTimeWindows.java          |   32 +-
 .../tasks/TestProcessingTimeService.java        |    2 +-
 .../api/operators/TestInternalTimerService.java |  238 ++
 .../TestProcessingTimeServiceTest.java          |    2 +-
 .../operators/windowing/CountTriggerTest.java   |  166 ++
 .../windowing/EventTimeSessionWindowsTest.java  |  179 ++
 .../windowing/EventTimeTriggerTest.java         |  153 ++
 .../operators/windowing/GlobalWindowsTest.java  |   59 +
 .../windowing/MergingWindowSetTest.java         |   25 +
 .../ProcessingTimeSessionWindowsTest.java       |  184 ++
 .../windowing/ProcessingTimeTriggerTest.java    |  134 +
 .../operators/windowing/PurgingTriggerTest.java |  149 ++
 .../windowing/SimpleTriggerTestHarness.java     |   41 +
 .../windowing/SlidingEventTimeWindowsTest.java  |  168 ++
 .../SlidingProcessingTimeWindowsTest.java       |  177 ++
 .../windowing/StreamRecordMatchers.java         |  179 ++
 .../operators/windowing/TriggerTestHarness.java |  369 +++
 .../windowing/TumblingEventTimeWindowsTest.java |  113 +
 .../TumblingProcessingTimeWindowsTest.java      |  129 +
 .../windowing/WindowOperatorContractTest.java   | 2495 ++++++++++++++++++
 .../operators/windowing/WindowOperatorTest.java |  171 --
 .../operators/windowing/WindowedValue.java      |   47 +
 .../windowing/WindowingTestHarnessTest.java     |  230 --
 .../util/AbstractStreamOperatorTestHarness.java |   20 +
 .../KeyedOneInputStreamOperatorTestHarness.java |   17 +
 .../flink/streaming/util/TestHarnessUtil.java   |   15 +-
 .../streaming/util/WindowingTestHarness.java    |  221 --
 35 files changed, 5209 insertions(+), 700 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index 9a9178a..9c16ed7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.state.heap;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListState;
@@ -505,4 +506,48 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			return stateSerializer;
 		}
 	}
+
+	/**
+	 * Returns the total number of state entries across all keys/namespaces.
+	 */
+	@VisibleForTesting
+	@SuppressWarnings("unchecked")
+	public int numStateEntries() {
+		int sum = 0;
+		for (StateTable<K, ?, ?> stateTable : stateTables.values()) {
+			for (Map namespaceMap : stateTable.getState()) {
+				if (namespaceMap == null) {
+					continue;
+				}
+				Map<?, Map> typedMap = (Map<?, Map>) namespaceMap;
+				for (Map entriesMap : typedMap.values()) {
+					sum += entriesMap.size();
+				}
+			}
+		}
+		return sum;
+	}
+
+	/**
+	 * Returns the total number of state entries across all keys for the given namespace.
+	 */
+	@VisibleForTesting
+	@SuppressWarnings("unchecked")
+	public <N> int numStateEntries(N namespace) {
+		int sum = 0;
+		for (StateTable<K, ?, ?> stateTable : stateTables.values()) {
+			for (Map namespaceMap : stateTable.getState()) {
+				if (namespaceMap == null) {
+					continue;
+				}
+				Map<?, Map> typedMap = (Map<?, Map>) namespaceMap;
+				Map singleNamespace = typedMap.get(namespace);
+				if (singleNamespace != null) {
+					sum += singleNamespace.size();
+				}
+			}
+		}
+		return sum;
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
index ac6adff..c267afc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
@@ -18,7 +18,12 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.junit.Test;
 
@@ -56,6 +61,44 @@ public class MemoryStateBackendTest extends StateBackendTestBase<MemoryStateBack
 	public void testReducingStateRestoreWithWrongSerializers() {}
 
 	@Test
+	@SuppressWarnings("unchecked")
+	public void testNumStateEntries() throws Exception {
+		KeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+
+		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class, null);
+		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		HeapKeyedStateBackend<Integer> heapBackend = (HeapKeyedStateBackend<Integer>) backend;
+
+		assertEquals(0, heapBackend.numStateEntries());
+
+		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		backend.setCurrentKey(0);
+		state.update("hello");
+		state.update("ciao");
+
+		assertEquals(1, heapBackend.numStateEntries());
+
+		backend.setCurrentKey(42);
+		state.update("foo");
+
+		assertEquals(2, heapBackend.numStateEntries());
+
+		backend.setCurrentKey(0);
+		state.clear();
+
+		assertEquals(1, heapBackend.numStateEntries());
+
+		backend.setCurrentKey(42);
+		state.clear();
+
+		assertEquals(0, heapBackend.numStateEntries());
+
+		backend.dispose();
+	}
+
+	@Test
 	public void testOversizedState() {
 		try {
 			MemoryStateBackend backend = new MemoryStateBackend(10);

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java
index e38f617..1703f6c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java
@@ -47,6 +47,10 @@ public class EventTimeSessionWindows extends MergingWindowAssigner<Object, TimeW
 	protected long sessionTimeout;
 
 	protected EventTimeSessionWindows(long sessionTimeout) {
+		if (sessionTimeout <= 0) {
+			throw new IllegalArgumentException("EventTimeSessionWindows parameters must satisfy 0 < size");
+		}
+
 		this.sessionTimeout = sessionTimeout;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
index 7ea3158..9e3846d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
+++ b/flink-streaming-java/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.annotation.Internal;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -70,7 +71,8 @@ public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
 	/**
 	 * A trigger that never fires, as default Trigger for GlobalWindows.
 	 */
-	private static class NeverTrigger extends Trigger<Object, GlobalWindow> {
+	@Internal
+	public static class NeverTrigger extends Trigger<Object, GlobalWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java
index 52d1c03..02c680e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/ProcessingTimeSessionWindows.java
@@ -47,6 +47,10 @@ public class ProcessingTimeSessionWindows extends MergingWindowAssigner<Object,
 	protected long sessionTimeout;
 
 	protected ProcessingTimeSessionWindows(long sessionTimeout) {
+		if (sessionTimeout <= 0) {
+			throw new IllegalArgumentException("ProcessingTimeSessionWindows parameters must satisfy 0 < size");
+		}
+
 		this.sessionTimeout = sessionTimeout;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingEventTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingEventTimeWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingEventTimeWindows.java
index 16171a0..ef6ed56 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingEventTimeWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingEventTimeWindows.java
@@ -55,6 +55,10 @@ public class SlidingEventTimeWindows extends WindowAssigner<Object, TimeWindow>
 	private final long offset;
 
 	protected SlidingEventTimeWindows(long size, long slide, long offset) {
+		if (offset < 0 || offset >= slide || size <= 0) {
+			throw new IllegalArgumentException("SlidingEventTimeWindows parameters must satisfy 0 <= offset < slide and size > 0");
+		}
+
 		this.size = size;
 		this.slide = slide;
 		this.offset = offset;
@@ -109,20 +113,18 @@ public class SlidingEventTimeWindows extends WindowAssigner<Object, TimeWindow>
 	}
 
 	/**
-	 *  Creates a new {@code SlidingEventTimeWindows} {@link WindowAssigner} that assigns
-	 *  elements to time windows based on the element timestamp and offset.
-	 *<p>
-	 *     For example, if you want window a stream by hour,but window begins at the 15th minutes
-	 *     of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
-	 *     time windows start at 0:15:00,1:15:00,2:15:00,etc.
-	 *</p>
+	 * Creates a new {@code SlidingEventTimeWindows} {@link WindowAssigner} that assigns
+	 * elements to time windows based on the element timestamp and offset.
+	 *
+	 * <p>For example, if you want window a stream by hour,but window begins at the 15th minutes
+	 * of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
+	 * time windows start at 0:15:00,1:15:00,2:15:00,etc.
+	 *
+	 * <p>Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
+	 * such as China which is using UTC+08:00,and you want a time window with size of one day,
+	 * and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
+	 * The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
 	 *
-	 * <p>
-	 *     Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
-	 *     such as China which is using UTC+08:00,and you want a time window with size of one day,
-	 *     and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
-	 *     The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
-	 * </p>
 	 * @param size The size of the generated windows.
 	 * @param slide  The slide interval of the generated windows.
 	 * @param offset The offset which window start would be shifted by.

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
index e03467f..c11045d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
@@ -51,7 +51,11 @@ public class SlidingProcessingTimeWindows extends WindowAssigner<Object, TimeWin
 
 	private final long slide;
 
-	private SlidingProcessingTimeWindows(long size, long slide, long offset){
+	private SlidingProcessingTimeWindows(long size, long slide, long offset) {
+		if (offset < 0 || offset >= slide || size <= 0) {
+			throw new IllegalArgumentException("SlidingProcessingTimeWindows parameters must satisfy 0 <= offset < slide and size > 0");
+		}
+
 		this.size = size;
 		this.slide = slide;
 		this.offset = offset;
@@ -101,20 +105,18 @@ public class SlidingProcessingTimeWindows extends WindowAssigner<Object, TimeWin
 	}
 
 	/**
-	 *  Creates a new {@code SlidingProcessingTimeWindows} {@link WindowAssigner} that assigns
-	 *  elements to time windows based on the element timestamp and offset.
-	 *<p>
-	 *     For example, if you want window a stream by hour,but window begins at the 15th minutes
-	 *     of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
-	 *     time windows start at 0:15:00,1:15:00,2:15:00,etc.
-	 *</p>
+	 * Creates a new {@code SlidingProcessingTimeWindows} {@link WindowAssigner} that assigns
+	 * elements to time windows based on the element timestamp and offset.
+	 *
+	 * <p>For example, if you want window a stream by hour,but window begins at the 15th minutes
+	 * of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
+	 * time windows start at 0:15:00,1:15:00,2:15:00,etc.
+	 *
+	 * <p>Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
+	 * such as China which is using UTC+08:00,and you want a time window with size of one day,
+	 * and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
+	 * The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
 	 *
-	 * <p>
-	 *     Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
-	 *     such as China which is using UTC+08:00,and you want a time window with size of one day,
-	 *     and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
-	 *     The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
-	 * </p>
 	 * @param size The size of the generated windows.
 	 * @param slide  The slide interval of the generated windows.
 	 * @param offset The offset which window start would be shifted by.

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingEventTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingEventTimeWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingEventTimeWindows.java
index b7fa343..d695a0c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingEventTimeWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingEventTimeWindows.java
@@ -51,7 +51,11 @@ public class TumblingEventTimeWindows extends WindowAssigner<Object, TimeWindow>
 
 	private final long offset;
 
-	protected TumblingEventTimeWindows(long size, long offset){
+	protected TumblingEventTimeWindows(long size, long offset) {
+		if (offset < 0 || offset >= size) {
+			throw new IllegalArgumentException("TumblingEventTimeWindows parameters must satisfy 0 <= offset < size");
+		}
+
 		this.size = size;
 		this.offset = offset;
 	}
@@ -68,10 +72,6 @@ public class TumblingEventTimeWindows extends WindowAssigner<Object, TimeWindow>
 		}
 	}
 
-	public long getSize() {
-		return size;
-	}
-
 	@Override
 	public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
 		return EventTimeTrigger.create();
@@ -94,26 +94,24 @@ public class TumblingEventTimeWindows extends WindowAssigner<Object, TimeWindow>
 	}
 
 	/**
-	 *  Creates a new {@code TumblingEventTimeWindows} {@link WindowAssigner} that assigns
-	 *  elements to time windows based on the element timestamp and offset.
-	 *<p>
-	 *     For example, if you want window a stream by hour,but window begins at the 15th minutes
-	 *     of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
-	 *     time windows start at 0:15:00,1:15:00,2:15:00,etc.
-	 *</p>
+	 * Creates a new {@code TumblingEventTimeWindows} {@link WindowAssigner} that assigns
+	 * elements to time windows based on the element timestamp and offset.
+	 *
+	 * <p>For example, if you want window a stream by hour,but window begins at the 15th minutes
+	 * of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
+	 * time windows start at 0:15:00,1:15:00,2:15:00,etc.
+	 *
+	 * <p>Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
+	 * such as China which is using UTC+08:00,and you want a time window with size of one day,
+	 * and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
+	 * The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
 	 *
-	 * <p>
-	 *     Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
-	 *     such as China which is using UTC+08:00,and you want a time window with size of one day,
-	 *     and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
-	 *     The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
-	 * </p>
 	 * @param size The size of the generated windows.
 	 * @param offset The offset which window start would be shifted by.
 	 * @return The time policy.
 	 */
 	public static TumblingEventTimeWindows of(Time size, Time offset) {
-		return new TumblingEventTimeWindows(size.toMilliseconds(), offset.toMilliseconds() % size.toMilliseconds());
+		return new TumblingEventTimeWindows(size.toMilliseconds(), offset.toMilliseconds());
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
index f4fb620..5b39fe0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
@@ -49,7 +49,11 @@ public class TumblingProcessingTimeWindows extends WindowAssigner<Object, TimeWi
 	private final long offset;
 
 
-	private TumblingProcessingTimeWindows(long size,long offset) {
+	private TumblingProcessingTimeWindows(long size, long offset) {
+		if (offset < 0 || offset >= size) {
+			throw new IllegalArgumentException("TumblingProcessingTimeWindows parameters must satisfy  0 <= offset < size");
+		}
+
 		this.size = size;
 		this.offset = offset;
 	}
@@ -87,26 +91,24 @@ public class TumblingProcessingTimeWindows extends WindowAssigner<Object, TimeWi
 	}
 
 	/**
-	 *  Creates a new {@code TumblingProcessingTimeWindows} {@link WindowAssigner} that assigns
-	 *  elements to time windows based on the element timestamp and offset.
-	 *<p>
-	 *     For example, if you want window a stream by hour,but window begins at the 15th minutes
-	 *     of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
-	 *     time windows start at 0:15:00,1:15:00,2:15:00,etc.
-	 *</p>
+	 * Creates a new {@code TumblingProcessingTimeWindows} {@link WindowAssigner} that assigns
+	 * elements to time windows based on the element timestamp and offset.
+	 *
+	 * <p>For example, if you want window a stream by hour,but window begins at the 15th minutes
+	 * of each hour, you can use {@code of(Time.hours(1),Time.minutes(15))},then you will get
+	 * time windows start at 0:15:00,1:15:00,2:15:00,etc.
+	 *
+	 * <p>Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
+	 * such as China which is using UTC+08:00,and you want a time window with size of one day,
+	 * and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
+	 * The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
 	 *
-	 * <p>
-	 *     Rather than that,if you are living in somewhere which is not using UTC�00:00 time,
-	 *     such as China which is using UTC+08:00,and you want a time window with size of one day,
-	 *     and window begins at every 00:00:00 of local time,you may use {@code of(Time.days(1),Time.hours(-8))}.
-	 *     The parameter of offset is {@code Time.hours(-8))} since UTC+08:00 is 8 hours earlier than UTC time.
-	 * </p>
 	 * @param size The size of the generated windows.
 	 * @param offset The offset which window start would be shifted by.
 	 * @return The time policy.
 	 */
 	public static TumblingProcessingTimeWindows of(Time size, Time offset) {
-		return new TumblingProcessingTimeWindows(size.toMilliseconds(), offset.toMilliseconds() % size.toMilliseconds());
+		return new TumblingProcessingTimeWindows(size.toMilliseconds(), offset.toMilliseconds());
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java
index b4e7e97..e5fcc1a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java
@@ -37,7 +37,7 @@ import java.util.concurrent.atomic.AtomicReference;
  * */
 public class TestProcessingTimeService extends ProcessingTimeService {
 
-	private volatile long currentTime = 0L;
+	private volatile long currentTime = Long.MIN_VALUE;
 
 	private volatile boolean isTerminated;
 	private volatile boolean isQuiesced;

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestInternalTimerService.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestInternalTimerService.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestInternalTimerService.java
new file mode 100644
index 0000000..a03a4c5
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TestInternalTimerService.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.operators;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * Implementation of {@link InternalTimerService} meant to use for testing.
+ */
+@Internal
+public class TestInternalTimerService<K, N> implements InternalTimerService<N> {
+
+	private long currentProcessingTime = Long.MIN_VALUE;
+
+	private long currentWatermark = Long.MIN_VALUE;
+
+	private final KeyContext keyContext;
+
+	/**
+	 * Processing time timers that are currently in-flight.
+	 */
+	private final PriorityQueue<Timer<K, N>> processingTimeTimersQueue;
+	private final Set<Timer<K, N>> processingTimeTimers;
+
+	/**
+	 * Current waiting watermark callbacks.
+	 */
+	private final Set<Timer<K, N>> watermarkTimers;
+	private final PriorityQueue<Timer<K, N>> watermarkTimersQueue;
+
+	public TestInternalTimerService(KeyContext keyContext) {
+		this.keyContext = keyContext;
+
+		watermarkTimers = new HashSet<>();
+		watermarkTimersQueue = new PriorityQueue<>(100);
+		processingTimeTimers = new HashSet<>();
+		processingTimeTimersQueue = new PriorityQueue<>(100);
+	}
+
+	@Override
+	public long currentProcessingTime() {
+		return currentProcessingTime;
+	}
+
+	@Override
+	public long currentWatermark() {
+		return currentWatermark;
+	}
+
+	@Override
+	public void registerProcessingTimeTimer(N namespace, long time) {
+		@SuppressWarnings("unchecked")
+		Timer<K, N> timer = new Timer<>(time, (K) keyContext.getCurrentKey(), namespace);
+		// make sure we only put one timer per key into the queue
+		if (processingTimeTimers.add(timer)) {
+			processingTimeTimersQueue.add(timer);
+		}
+	}
+
+	@Override
+	public void registerEventTimeTimer(N namespace, long time) {
+		@SuppressWarnings("unchecked")
+		Timer<K, N> timer = new Timer<>(time, (K) keyContext.getCurrentKey(), namespace);
+		if (watermarkTimers.add(timer)) {
+			watermarkTimersQueue.add(timer);
+		}
+	}
+
+	@Override
+	public void deleteProcessingTimeTimer(N namespace, long time) {
+		@SuppressWarnings("unchecked")
+		Timer<K, N> timer = new Timer<>(time, (K) keyContext.getCurrentKey(), namespace);
+
+		if (processingTimeTimers.remove(timer)) {
+			processingTimeTimersQueue.remove(timer);
+		}
+	}
+
+	@Override
+	public void deleteEventTimeTimer(N namespace, long time) {
+		@SuppressWarnings("unchecked")
+		Timer<K, N> timer = new Timer<>(time, (K) keyContext.getCurrentKey(), namespace);
+		if (watermarkTimers.remove(timer)) {
+			watermarkTimersQueue.remove(timer);
+		}
+	}
+
+	public Collection<Timer<K, N>> advanceProcessingTime(long time) throws Exception {
+		List<Timer<K, N>> result = new ArrayList<>();
+
+		Timer<K, N> timer = processingTimeTimersQueue.peek();
+
+		while (timer != null && timer.timestamp <= time) {
+			processingTimeTimers.remove(timer);
+			processingTimeTimersQueue.remove();
+			result.add(timer);
+			timer = processingTimeTimersQueue.peek();
+		}
+
+		currentProcessingTime = time;
+		return result;
+	}
+
+	public Collection<Timer<K, N>> advanceWatermark(long time) throws Exception {
+		List<Timer<K, N>> result = new ArrayList<>();
+
+		Timer<K, N> timer = watermarkTimersQueue.peek();
+
+		while (timer != null && timer.timestamp <= time) {
+			watermarkTimers.remove(timer);
+			watermarkTimersQueue.remove();
+			result.add(timer);
+			timer = watermarkTimersQueue.peek();
+		}
+
+		currentWatermark = time;
+		return result;
+	}
+
+	/**
+	 * Internal class for keeping track of in-flight timers.
+	 */
+	public static class Timer<K, N> implements Comparable<Timer<K, N>> {
+		private final long timestamp;
+		private final K key;
+		private final N namespace;
+
+		public Timer(long timestamp, K key, N namespace) {
+			this.timestamp = timestamp;
+			this.key = key;
+			this.namespace = namespace;
+		}
+
+		public long getTimestamp() {
+			return timestamp;
+		}
+
+		public K getKey() {
+			return key;
+		}
+
+		public N getNamespace() {
+			return namespace;
+		}
+
+		@Override
+		public int compareTo(Timer<K, N> o) {
+			return Long.compare(this.timestamp, o.timestamp);
+		}
+
+		@Override
+		public boolean equals(Object o) {
+			if (this == o) {
+				return true;
+			}
+			if (o == null || getClass() != o.getClass()){
+				return false;
+			}
+
+			Timer<?, ?> timer = (Timer<?, ?>) o;
+
+			return timestamp == timer.timestamp
+					&& key.equals(timer.key)
+					&& namespace.equals(timer.namespace);
+
+		}
+
+		@Override
+		public int hashCode() {
+			int result = (int) (timestamp ^ (timestamp >>> 32));
+			result = 31 * result + key.hashCode();
+			result = 31 * result + namespace.hashCode();
+			return result;
+		}
+
+		@Override
+		public String toString() {
+			return "Timer{" +
+					"timestamp=" + timestamp +
+					", key=" + key +
+					", namespace=" + namespace +
+					'}';
+		}
+	}
+
+	public int numProcessingTimeTimers() {
+		return processingTimeTimers.size();
+	}
+
+	public int numEventTimeTimers() {
+		return watermarkTimers.size();
+	}
+
+	public int numProcessingTimeTimers(N namespace) {
+		int count = 0;
+		for (Timer<K, N> timer : processingTimeTimers) {
+			if (timer.getNamespace().equals(namespace)) {
+				count++;
+			}
+		}
+
+		return count;
+	}
+
+	public int numEventTimeTimers(N namespace) {
+		int count = 0;
+		for (Timer<K, N> timer : watermarkTimers) {
+			if (timer.getNamespace().equals(namespace)) {
+				count++;
+			}
+		}
+
+		return count;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java
index 2903758..4d24b82 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java
@@ -52,7 +52,7 @@ public class TestProcessingTimeServiceTest {
 
 		testHarness.invoke();
 
-		assertEquals(testHarness.getProcessingTimeService().getCurrentProcessingTime(), 0);
+		assertEquals(Long.MIN_VALUE, testHarness.getProcessingTimeService().getCurrentProcessingTime());
 
 		tp.setCurrentTime(11);
 		assertEquals(testHarness.getProcessingTimeService().getCurrentProcessingTime(), 11);

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java
new file mode 100644
index 0000000..16e353b
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link CountTrigger}.
+ */
+public class CountTriggerTest {
+
+	/**
+	 * Verify that state of separate windows does not leak into other windows.
+	 */
+	@Test
+	public void testWindowSeparationAndFiring() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(CountTrigger.<TimeWindow>of(3), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		// shouldn't have any timers
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+
+		assertEquals(2, testHarness.numStateEntries());
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		// right now, CountTrigger will clear it's state in onElement when firing
+		// ideally, this should be moved to onFire()
+		assertEquals(1, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		// now all state should be gone
+		assertEquals(0, testHarness.numStateEntries());
+	}
+
+	/**
+	 * Verify that clear() does not leak across windows.
+	 */
+	@Test
+	public void testClear() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(CountTrigger.<TimeWindow>of(3), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		// shouldn't have any timers
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+
+		assertEquals(2, testHarness.numStateEntries());
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(2, 4));
+
+		assertEquals(1, testHarness.numStateEntries());
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(0, 2));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(2, 4)));
+	}
+
+	@Test
+	public void testMergingWindows() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(CountTrigger.<TimeWindow>of(3), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(4, 6)));
+
+		// shouldn't have any timers
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+
+		assertEquals(3, testHarness.numStateEntries());
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(2, 4)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(4, 6)));
+
+		testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)));
+
+		assertEquals(2, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(2, 4)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 4)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(4, 6)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 4)));
+
+		assertEquals(1, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(0, 4)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(4, 6)));
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(4, 6)));
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(4, 6)));
+
+		assertEquals(0, testHarness.numStateEntries());
+	}
+
+	@Test
+	public void testMergeSubsumingWindow() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(CountTrigger.<TimeWindow>of(3), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(4, 6)));
+
+		// shouldn't have any timers
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+
+		assertEquals(2, testHarness.numStateEntries());
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(2, 4)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(4, 6)));
+
+		testHarness.mergeWindows(new TimeWindow(0, 8), Lists.newArrayList(new TimeWindow(2, 4), new TimeWindow(4, 6)));
+
+		assertEquals(1, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(2, 4)));
+		assertEquals(0, testHarness.numStateEntries(new TimeWindow(4, 6)));
+		assertEquals(1, testHarness.numStateEntries(new TimeWindow(0, 8)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 8)));
+
+		assertEquals(0, testHarness.numStateEntries());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java
new file mode 100644
index 0000000..a46572b
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+import java.util.Collection;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.anyCollection;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for {@link EventTimeSessionWindows}
+ */
+public class EventTimeSessionWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		final int SESSION_GAP = 5000;
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.milliseconds(SESSION_GAP));
+
+		assertThat(assigner.assignWindows("String", 0L, mockContext), contains(timeWindow(0, 0 + SESSION_GAP)));
+		assertThat(assigner.assignWindows("String", 4999L, mockContext), contains(timeWindow(4999, 4999 + SESSION_GAP)));
+		assertThat(assigner.assignWindows("String", 5000L, mockContext), contains(timeWindow(5000, 5000 + SESSION_GAP)));
+	}
+
+	@Test
+	public void testMergeSinglePointWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 0)), callback);
+
+		verify(callback, never()).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeSingleWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 1)), callback);
+
+		verify(callback, never()).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeConsecutiveWindows() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(
+				Lists.newArrayList(
+						new TimeWindow(0, 1),
+						new TimeWindow(1, 2),
+						new TimeWindow(2, 3),
+						new TimeWindow(4, 5),
+						new TimeWindow(5, 6)),
+				callback);
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(0, 1), new TimeWindow(1, 2), new TimeWindow(2, 3))),
+				eq(new TimeWindow(0, 3)));
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(4, 5), new TimeWindow(5, 6))),
+				eq(new TimeWindow(4, 6)));
+
+		verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeCoveringWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(
+				Lists.newArrayList(
+						new TimeWindow(1, 1),
+						new TimeWindow(0, 2),
+						new TimeWindow(4, 7),
+						new TimeWindow(5, 6)),
+				callback);
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(1, 1), new TimeWindow(0, 2))),
+				eq(new TimeWindow(0, 2)));
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(5, 6), new TimeWindow(4, 7))),
+				eq(new TimeWindow(4, 7)));
+
+		verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		final int SESSION_GAP = 5000;
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.seconds(SESSION_GAP / 1000));
+
+		assertThat(assigner.assignWindows("String", 0L, mockContext), contains(timeWindow(0, 0 + SESSION_GAP)));
+		assertThat(assigner.assignWindows("String", 4999L, mockContext), contains(timeWindow(4999, 4999 + SESSION_GAP)));
+		assertThat(assigner.assignWindows("String", 5000L, mockContext), contains(timeWindow(5000, 5000 + SESSION_GAP)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			EventTimeSessionWindows.withGap(Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 < size"));
+		}
+
+		try {
+			EventTimeSessionWindows.withGap(Time.seconds(0));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 < size"));
+		}
+
+	}
+
+	@Test
+	public void testProperties() {
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.seconds(5));
+
+		assertTrue(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(EventTimeTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java
new file mode 100644
index 0000000..2d93ac0
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link EventTimeTrigger}.
+ */
+public class EventTimeTriggerTest {
+
+	/**
+	 * Verify that state of separate windows does not leak into other windows.
+	 */
+	@Test
+	public void testWindowSeparationAndFiring() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(EventTimeTrigger.create(), new TimeWindow.Serializer());
+
+		// inject several elements
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(2, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(2, new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+
+	/**
+	 * Verify that late elements trigger immediately and also that we don't set a timer
+	 * for those.
+	 */
+	@Test
+	public void testLateElementTriggersImmediately() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(EventTimeTrigger.create(), new TimeWindow.Serializer());
+
+		testHarness.advanceWatermark(2);
+
+		assertEquals(TriggerResult.FIRE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+
+
+	/**
+	 * Verify that clear() does not leak across windows.
+	 */
+	@Test
+	public void testClear() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(EventTimeTrigger.create(), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(2, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(2, 4));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(0, 2));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+
+	@Test
+	public void testMergingWindows() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(EventTimeTrigger.create(), new TimeWindow.Serializer());
+
+		assertTrue(EventTimeTrigger.create().canMerge());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(2, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numEventTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(2, 4)));
+		assertEquals(1, testHarness.numEventTimeTimers(new TimeWindow(0, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceWatermark(4, new TimeWindow(0, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/GlobalWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/GlobalWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/GlobalWindowsTest.java
new file mode 100644
index 0000000..37fad7e
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/GlobalWindowsTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for {@link GlobalWindows}
+ */
+public class GlobalWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		GlobalWindows assigner = GlobalWindows.create();
+
+		assertThat(assigner.assignWindows("String", 0L, mockContext), contains(GlobalWindow.get()));
+		assertThat(assigner.assignWindows("String", 4999L, mockContext), contains(GlobalWindow.get()));
+		assertThat(assigner.assignWindows("String", 5000L, mockContext), contains(GlobalWindow.get()));
+	}
+
+	@Test
+	public void testProperties() {
+		GlobalWindows assigner = GlobalWindows.create();
+
+		assertFalse(assigner.isEventTime());
+		assertEquals(new GlobalWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(GlobalWindows.NeverTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
index 46169a8..aa9cb91 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
@@ -290,6 +290,31 @@ public class MergingWindowSetTest {
 	}
 
 	/**
+	 * Test adding a new window that is identical to an existing window. This should not cause
+	 * a merge.
+	 */
+	@Test
+	public void testAddingIdenticalWindows() throws Exception {
+		@SuppressWarnings("unchecked")
+		ListState<Tuple2<TimeWindow, TimeWindow>> mockState = mock(ListState.class);
+
+		MergingWindowSet<TimeWindow> windowSet = new MergingWindowSet<>(EventTimeSessionWindows.withGap(Time.milliseconds(3)), mockState);
+
+		TestingMergeFunction mergeFunction = new TestingMergeFunction();
+
+		mergeFunction.reset();
+		assertEquals(new TimeWindow(1, 2), windowSet.addWindow(new TimeWindow(1, 2), mergeFunction));
+		assertFalse(mergeFunction.hasMerged());
+		assertEquals(new TimeWindow(1, 2), windowSet.getStateWindow(new TimeWindow(1, 2)));
+
+		mergeFunction.reset();
+		assertEquals(new TimeWindow(1, 2), windowSet.addWindow(new TimeWindow(1, 2), mergeFunction));
+		assertFalse(mergeFunction.hasMerged());
+		assertEquals(new TimeWindow(1, 2), windowSet.getStateWindow(new TimeWindow(1, 2)));
+	}
+
+
+	/**
 	 * Test merging of a large new window that covers multiple existing windows.
 	 */
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java
new file mode 100644
index 0000000..461b5fc
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ExecutionConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.ProcessingTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+import java.util.Collection;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.timeWindow;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.anyCollection;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for {@link ProcessingTimeSessionWindows}
+ */
+public class ProcessingTimeSessionWindowsTest extends TestLogger {
+
+	@Test
+	public void testWindowAssignment() {
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(0L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(0, 5000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(4999L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(4999, 9999)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testMergeSinglePointWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 0)), callback);
+
+		verify(callback, never()).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeSingleWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(Lists.newArrayList(new TimeWindow(0, 1)), callback);
+
+		verify(callback, never()).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeConsecutiveWindows() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(
+				Lists.newArrayList(
+						new TimeWindow(0, 1),
+						new TimeWindow(1, 2),
+						new TimeWindow(2, 3),
+						new TimeWindow(4, 5),
+						new TimeWindow(5, 6)),
+				callback);
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(0, 1), new TimeWindow(1, 2), new TimeWindow(2, 3))),
+				eq(new TimeWindow(0, 3)));
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(4, 5), new TimeWindow(5, 6))),
+				eq(new TimeWindow(4, 6)));
+
+		verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testMergeCoveringWindow() {
+		MergingWindowAssigner.MergeCallback callback = mock(MergingWindowAssigner.MergeCallback.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.milliseconds(5000));
+
+		assigner.mergeWindows(
+				Lists.newArrayList(
+						new TimeWindow(1, 1),
+						new TimeWindow(0, 2),
+						new TimeWindow(4, 7),
+						new TimeWindow(5, 6)),
+				callback);
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(1, 1), new TimeWindow(0, 2))),
+				eq(new TimeWindow(0, 2)));
+
+		verify(callback, times(1)).merge(
+				(Collection<TimeWindow>) argThat(containsInAnyOrder(new TimeWindow(5, 6), new TimeWindow(4, 7))),
+				eq(new TimeWindow(4, 7)));
+
+		verify(callback, times(2)).merge(anyCollection(), Matchers.anyObject());
+	}
+
+	@Test
+	public void testTimeUnits() {
+		// sanity check with one other time unit
+
+		WindowAssigner.WindowAssignerContext mockContext =
+				mock(WindowAssigner.WindowAssignerContext.class);
+
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.seconds(5));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(0L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(0, 5000)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(4999L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(4999, 9999)));
+
+		when(mockContext.getCurrentProcessingTime()).thenReturn(5000L);
+		assertThat(assigner.assignWindows("String", Long.MIN_VALUE, mockContext), contains(timeWindow(5000, 10000)));
+	}
+
+	@Test
+	public void testInvalidParameters() {
+		try {
+			ProcessingTimeSessionWindows.withGap(Time.seconds(-1));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 < size"));
+		}
+
+		try {
+			ProcessingTimeSessionWindows.withGap(Time.seconds(0));
+			fail("should fail");
+		} catch (IllegalArgumentException e) {
+			assertThat(e.toString(), containsString("0 < size"));
+		}
+
+	}
+
+	@Test
+	public void testProperties() {
+		ProcessingTimeSessionWindows assigner = ProcessingTimeSessionWindows.withGap(Time.seconds(5));
+
+		assertFalse(assigner.isEventTime());
+		assertEquals(new TimeWindow.Serializer(), assigner.getWindowSerializer(new ExecutionConfig()));
+		assertThat(assigner.getDefaultTrigger(mock(StreamExecutionEnvironment.class)), instanceOf(ProcessingTimeTrigger.class));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java
new file mode 100644
index 0000000..a0c2347
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link ProcessingTimeTrigger}.
+ */
+public class ProcessingTimeTriggerTest {
+
+	/**
+	 * Verify that state of separate windows does not leak into other windows.
+	 */
+	@Test
+	public void testWindowSeparationAndFiring() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(ProcessingTimeTrigger.create(), new TimeWindow.Serializer());
+
+		// inject several elements
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(2, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(2, new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+
+	/**
+	 * Verify that clear() does not leak across windows.
+	 */
+	@Test
+	public void testClear() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(ProcessingTimeTrigger.create(), new TimeWindow.Serializer());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(2, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(2, 4));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.clearTriggerState(new TimeWindow(0, 2));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+	}
+
+	@Test
+	public void testMergingWindows() throws Exception {
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(ProcessingTimeTrigger.create(), new TimeWindow.Serializer());
+
+		assertTrue(ProcessingTimeTrigger.create().canMerge());
+
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(2, testHarness.numProcessingTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+
+		testHarness.mergeWindows(new TimeWindow(0, 4), Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numEventTimeTimers());
+		assertEquals(1, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numProcessingTimeTimers(new TimeWindow(2, 4)));
+		assertEquals(1, testHarness.numProcessingTimeTimers(new TimeWindow(0, 4)));
+
+		assertEquals(TriggerResult.FIRE, testHarness.advanceProcessingTime(4, new TimeWindow(0, 4)));
+
+		assertEquals(0, testHarness.numStateEntries());
+		assertEquals(0, testHarness.numProcessingTimeTimers());
+		assertEquals(0, testHarness.numEventTimeTimers());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/bb8586e5/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PurgingTriggerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PurgingTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PurgingTriggerTest.java
new file mode 100644
index 0000000..4302d4d
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PurgingTriggerTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.lang.reflect.Method;
+import java.util.Collections;
+
+import static org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorContractTest.anyOnMergeContext;
+import static org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorContractTest.anyTimeWindow;
+import static org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorContractTest.anyTriggerContext;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for {@link PurgingTrigger}.
+ */
+public class PurgingTriggerTest {
+
+	/**
+	 * Check if {@link PurgingTrigger} implements all methods of {@link Trigger}, as a sanity
+	 * check.
+	 */
+	@Test
+	public void testAllMethodsImplemented() throws NoSuchMethodException {
+		for (Method triggerMethod : Trigger.class.getDeclaredMethods()) {
+
+			// try retrieving the method, this will throw an exception if we can't find it
+			PurgingTrigger.class.getDeclaredMethod(triggerMethod.getName(),triggerMethod.getParameterTypes());
+		}
+	}
+
+	@Test
+	public void testForwarding() throws Exception {
+		Trigger<Object, TimeWindow> mockTrigger = mock(Trigger.class);
+
+		TriggerTestHarness<Object, TimeWindow> testHarness =
+				new TriggerTestHarness<>(PurgingTrigger.of(mockTrigger), new TimeWindow.Serializer());
+
+		when(mockTrigger.onElement(Matchers.anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		assertEquals(TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+
+		when(mockTrigger.onElement(Matchers.anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+
+		when(mockTrigger.onElement(Matchers.anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+
+		when(mockTrigger.onElement(Matchers.anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		assertEquals(TriggerResult.PURGE, testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2)));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+
+				// register some timers that we can step through to call onEventTime several
+				// times in a row
+				context.registerEventTimeTimer(1);
+				context.registerEventTimeTimer(2);
+				context.registerEventTimeTimer(3);
+				context.registerEventTimeTimer(4);
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		// set up our timers
+		testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2));
+
+		assertEquals(4, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		assertEquals(TriggerResult.CONTINUE, testHarness.advanceWatermark(1, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.advanceWatermark(2, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.advanceWatermark(3, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		assertEquals(TriggerResult.PURGE, testHarness.advanceWatermark(4, new TimeWindow(0, 2)));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+
+				// register some timers that we can step through to call onEventTime several
+				// times in a row
+				context.registerProcessingTimeTimer(1);
+				context.registerProcessingTimeTimer(2);
+				context.registerProcessingTimeTimer(3);
+				context.registerProcessingTimeTimer(4);
+				return TriggerResult.CONTINUE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		// set up our timers
+		testHarness.processElement(new StreamRecord<Object>(1), new TimeWindow(0, 2));
+
+		assertEquals(4, testHarness.numProcessingTimeTimers(new TimeWindow(0, 2)));
+		assertEquals(0, testHarness.numEventTimeTimers(new TimeWindow(0, 2)));
+
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		assertEquals(TriggerResult.CONTINUE, testHarness.advanceProcessingTime(1, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.advanceProcessingTime(2, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		assertEquals(TriggerResult.FIRE_AND_PURGE, testHarness.advanceProcessingTime(3, new TimeWindow(0, 2)));
+
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		assertEquals(TriggerResult.PURGE, testHarness.advanceProcessingTime(4, new TimeWindow(0, 2)));
+
+		testHarness.mergeWindows(new TimeWindow(0, 2), Collections.singletonList(new TimeWindow(0, 1)));
+		verify(mockTrigger, times(1)).onMerge(anyTimeWindow(), anyOnMergeContext());
+
+		testHarness.clearTriggerState(new TimeWindow(0, 2));
+		verify(mockTrigger, times(1)).clear(eq(new TimeWindow(0, 2)), anyTriggerContext());
+	}
+
+}


[05/10] flink git commit: [hotfix] Use correct ClassLoader in WindowOperatorMigrationTest

Posted by al...@apache.org.
[hotfix] Use correct ClassLoader in WindowOperatorMigrationTest


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

Branch: refs/heads/release-1.2
Commit: 704b4116c3d5578049a3ea236a0d67f9ae47d443
Parents: 1d362f5
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Jan 13 14:22:39 2017 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Jan 23 14:53:22 2017 +0100

----------------------------------------------------------------------
 .../runtime/operators/windowing/WindowOperatorMigrationTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/704b4116/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
index 429ffbe..7a356cf 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
@@ -73,7 +73,7 @@ import static org.junit.Assert.fail;
 public class WindowOperatorMigrationTest {
 
 	private static String getResourceFilename(String filename) {
-		ClassLoader cl = WindowOperatorTest.class.getClassLoader();
+		ClassLoader cl = WindowOperatorMigrationTest.class.getClassLoader();
 		URL resource = cl.getResource(filename);
 		if (resource == null) {
 			throw new NullPointerException("Missing snapshot resource.");