You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/07/29 18:55:56 UTC

[GitHub] kl0u closed pull request #2415: [FLINK-4407] Implement the trigger DSL

kl0u closed pull request #2415: [FLINK-4407] Implement the trigger DSL
URL: https://github.com/apache/flink/pull/2415
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
index 483c9541662..026b4c5916f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
@@ -331,4 +331,40 @@ private void readObject(final ObjectInputStream in) throws IOException, ClassNot
 			defaultValue = null;
 		}
 	}
+
+	/**
+	 * Creates a new {@link StateDescriptor} of the same type as the argument <tt>stateDescriptor</tt> passed as an argument, but with the name
+	 * updated with the provided <tt>suffix</tt>. This method should be updated every time there is a change in the format of the state descriptors.
+	 *
+	 * @param stateDescriptor the original state descriptor to copy.
+	 * @param suffix the suffix to be appended in the name.
+	 * @return The resulting state descriptor.
+	 */
+	@SuppressWarnings("unchecked")
+	public static <S extends State> StateDescriptor<S, ?> translateStateDescriptorWithSuffix(StateDescriptor<S, ?> stateDescriptor, String suffix) {
+		String newName = stateDescriptor.getName().endsWith(suffix) ?
+			stateDescriptor.getName() :
+			stateDescriptor.getName() +"_"+ suffix;
+
+		StateDescriptor<S, ?> newStateDescriptor;
+		if (stateDescriptor instanceof ValueStateDescriptor) {
+			newStateDescriptor = new ValueStateDescriptor(newName, stateDescriptor.getSerializer(), stateDescriptor.getDefaultValue());
+		} else if (stateDescriptor instanceof ListStateDescriptor) {
+			newStateDescriptor = new ListStateDescriptor(newName, stateDescriptor.getSerializer());
+		} else if (stateDescriptor instanceof ReducingStateDescriptor) {
+			newStateDescriptor = new ReducingStateDescriptor(newName,
+				((ReducingStateDescriptor) stateDescriptor).getReduceFunction(), stateDescriptor.getSerializer());
+		} else if (stateDescriptor instanceof FoldingStateDescriptor) {
+			newStateDescriptor = new FoldingStateDescriptor(newName, stateDescriptor.getDefaultValue(),
+				((FoldingStateDescriptor) stateDescriptor).getFoldFunction(), stateDescriptor.getSerializer());
+		} else {
+			throw new RuntimeException("Unknown state descriptor: " + stateDescriptor);
+		}
+
+		// update also the queryable state name
+		String newQueryableStateName = stateDescriptor.getQueryableStateName() +"_"+ suffix;
+		newStateDescriptor.setQueryable(newQueryableStateName);
+
+		return newStateDescriptor;
+	}
 }
diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
index d7c70147e02..4d0f0c02320 100644
--- a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
+++ b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
@@ -32,6 +32,7 @@ import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
 import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
 
 import scala.language.postfixOps
 import scala.util.Random
@@ -106,7 +107,7 @@ object TopSpeedWindowing {
       .keyBy("carId")
       .window(GlobalWindows.create)
       .evictor(TimeEvictor.of(Time.of(evictionSec * 1000, TimeUnit.MILLISECONDS)))
-      .trigger(DeltaTrigger.of(triggerMeters, new DeltaFunction[CarEvent] {
+      .trigger(DeltaTrigger.of[CarEvent, GlobalWindow](triggerMeters, new DeltaFunction[CarEvent] {
         def getDelta(oldSp: CarEvent, newSp: CarEvent): Double = newSp.distance - oldSp.distance
       }, cars.getType().createSerializer(env.getConfig)))
 //      .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index 6b09f3c4abe..b4aff01bad3 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -27,6 +27,7 @@
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -44,6 +45,8 @@
 import org.apache.flink.streaming.api.windowing.evictors.Evictor;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTriggerRunner;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
 import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
@@ -51,6 +54,7 @@
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueAllWindowFunction;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.Preconditions;
 
 /**
  * A {@code AllWindowedStream} represents a data stream where the stream of
@@ -112,6 +116,28 @@ public AllWindowedStream(DataStream<T> input,
 		return this;
 	}
 
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	@PublicEvolving
+	public AllWindowedStream<T, W> trigger(DslTrigger<? super T, ? super W> trigger) {
+		if (windowAssigner instanceof MergingWindowAssigner && !trigger.canMerge()) {
+			throw new UnsupportedOperationException("A merging window assigner cannot be used with a trigger that does not support merging.");
+		}
+
+		this.trigger = new DslTriggerRunner<>(trigger);
+		return this;
+	}
+
+	private Trigger<? super T, ? super W> buildTrigger(TypeSerializer<W> windowSerializer) {
+		Preconditions.checkNotNull(this.trigger, "The trigger has not been initialized.");
+		if (trigger instanceof DslTriggerRunner) {
+			DslTriggerRunner runner = (DslTriggerRunner) trigger;
+			runner.createTriggerTree(windowSerializer, allowedLateness);
+		}
+		return trigger;
+	}
+
 	/**
 	 * Sets the time by which elements are allowed to be late. Elements that
 	 * arrive behind the watermark by more than the specified time will be dropped.
@@ -266,6 +292,9 @@ public AllWindowedStream(DataStream<T> input,
 		String opName;
 		KeySelector<T, Byte> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		WindowOperator<Byte, T, Iterable<T>, R, W> operator;
 
 		if (evictor != null) {
@@ -276,7 +305,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -293,7 +322,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -354,6 +383,9 @@ public AllWindowedStream(DataStream<T> input,
 		String opName;
 		KeySelector<T, Byte> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		OneInputStreamOperator<T, R> operator;
 
 		if (evictor != null) {
@@ -364,7 +396,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -382,7 +414,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -447,6 +479,9 @@ public AllWindowedStream(DataStream<T> input,
 		String opName;
 		KeySelector<T, Byte> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		OneInputStreamOperator<T, R> operator;
 
 		if (evictor != null) {
@@ -458,7 +493,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -475,7 +510,7 @@ public AllWindowedStream(DataStream<T> input,
 
 			operator =
 				new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index ae986190440..392b974be2b 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -28,6 +28,7 @@
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -48,6 +49,8 @@
 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.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTriggerRunner;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
@@ -58,6 +61,7 @@
 import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
+import org.apache.flink.util.Preconditions;
 
 /**
  * A {@code WindowedStream} represents a data stream where elements are grouped by
@@ -123,6 +127,28 @@ public WindowedStream(KeyedStream<T, K> input,
 		return this;
 	}
 
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	@PublicEvolving
+	public WindowedStream<T, K, W> trigger(DslTrigger<? super T, ? super W> trigger) {
+		if (windowAssigner instanceof MergingWindowAssigner && !trigger.canMerge()) {
+			throw new UnsupportedOperationException("A merging window assigner cannot be used with a trigger that does not support merging.");
+		}
+
+		this.trigger = new DslTriggerRunner<>(trigger);
+		return this;
+	}
+
+	private Trigger<? super T, ? super W> buildTrigger(TypeSerializer<W> windowSerializer) {
+		Preconditions.checkNotNull(this.trigger, "The trigger has not been initialized.");
+		if (trigger instanceof DslTriggerRunner) {
+			DslTriggerRunner runner = (DslTriggerRunner) trigger;
+			runner.createTriggerTree(windowSerializer, allowedLateness);
+		}
+		return trigger;
+	}
+
 	/**
 	 * Sets the time by which elements are allowed to be late. Elements that
 	 * arrive behind the watermark by more than the specified time will be dropped.
@@ -287,6 +313,9 @@ public WindowedStream(KeyedStream<T, K> input,
 		String opName;
 		KeySelector<T, K> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		WindowOperator<K, T, Iterable<T>, R, W> operator;
 
 		if (evictor != null) {
@@ -297,7 +326,7 @@ public WindowedStream(KeyedStream<T, K> input,
 
 			operator =
 				new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -314,7 +343,7 @@ public WindowedStream(KeyedStream<T, K> input,
 
 			operator =
 				new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -375,6 +404,9 @@ public WindowedStream(KeyedStream<T, K> input,
 		String opName;
 		KeySelector<T, K> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		OneInputStreamOperator<T, R> operator;
 
 		if (evictor != null) {
@@ -385,7 +417,7 @@ public WindowedStream(KeyedStream<T, K> input,
 
 			operator =
 				new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -403,7 +435,7 @@ public WindowedStream(KeyedStream<T, K> input,
 
 			operator =
 				new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					windowSerializer,
 					keySel,
 					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 					stateDesc,
@@ -468,6 +500,9 @@ public WindowedStream(KeyedStream<T, K> input,
 		String opName;
 		KeySelector<T, K> keySel = input.getKeySelector();
 
+		TypeSerializer<W> windowSerializer = windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig());
+		this.trigger = buildTrigger(windowSerializer);
+
 		OneInputStreamOperator<T, R> operator;
 
 		if (evictor != null) {
@@ -478,7 +513,7 @@ public WindowedStream(KeyedStream<T, K> input,
 			opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + evictor + ", " + udfName + ")";
 
 			operator = new EvictingWindowOperator<>(windowAssigner,
-				windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+				windowSerializer,
 				keySel,
 				input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 				stateDesc,
@@ -494,7 +529,7 @@ public WindowedStream(KeyedStream<T, K> input,
 			opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")";
 
 			operator = new WindowOperator<>(windowAssigner,
-				windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+				windowSerializer,
 				keySel,
 				input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
 				stateDesc,
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
index 46080ff1bc5..c780bfc09d6 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
@@ -85,6 +85,20 @@ public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) t
 		return TriggerResult.CONTINUE;
 	}
 
+	@Override
+	public void onFire(Window window, TriggerContext ctx) throws Exception {
+		// do nothing.
+		//
+		// Even if we fire at the end or not, the state has already been cleared in the onEventTime(),
+		// so there is nothing to do here. We do the cleanup in the onEventTime, as before, and not here,
+		// as in other triggers. This is because i) we assume that existing triggers are not combinable,
+		// so whenever this trigger proposes to fire, it will fire and ii) if somebody in the future tries
+		// to combine this with another trigger, then cleaning the state onEventTime gives the
+		// opportunity to the trigger to reset the timer for a future potential firing. If not, then the
+		// timer would be stuck to the same initial value, as in the onEventTime we only check for equality,
+		// and onElement we check if the state is null.
+	}
+
 	@Override
 	public void clear(W window, TriggerContext ctx) throws Exception {
 		ReducingState<Long> fireTimestamp = ctx.getPartitionedState(stateDesc);
@@ -106,7 +120,7 @@ public TriggerResult onMerge(W window, OnMergeContext ctx) {
 
 	@Override
 	public String toString() {
-		return "ContinuousProcessingTimeTrigger(" + interval + ")";
+		return "ContinuousEventTimeTrigger(" + interval + ")";
 	}
 
 	@VisibleForTesting
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
index 287d3df6adc..c05109a3410 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
@@ -70,6 +70,20 @@ public TriggerResult onEventTime(long time, W window, TriggerContext ctx) throws
 		return TriggerResult.CONTINUE;
 	}
 
+	@Override
+	public void onFire(W window, TriggerContext ctx) throws Exception {
+		// do nothing.
+		//
+		// Even if we fire at the end or not, the state has already been cleared in the onProcessingTime(),
+		// so there is nothing to do here. We do the cleanup in the onProcessingTime, as before, and not here,
+		// as in other triggers. This is because i) we assume that existing triggers are not combinable,
+		// so whenever this trigger proposes to fire, it will fire and ii) if somebody in the future tries
+		// to combine this with another trigger, then cleaning the state onProcessingTime gives the
+		// opportunity to the trigger to reset the timer for a future potential firing. If not, then the
+		// timer would be stuck to the same initial value, as in the onProcessingTime we only check for equality,
+		// and onElement we check if the state is null.
+	}
+
 	@Override
 	public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception {
 		ReducingState<Long> fireTimestamp = ctx.getPartitionedState(stateDesc);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
index 86c5c4cf363..02394509324 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
@@ -48,11 +48,7 @@ private CountTrigger(long maxCount) {
 	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) throws Exception {
 		ReducingState<Long> count = ctx.getPartitionedState(stateDesc);
 		count.add(1L);
-		if (count.get() >= maxCount) {
-			count.clear();
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
+		return (count.get() >= maxCount) ? TriggerResult.FIRE : TriggerResult.CONTINUE;
 	}
 
 	@Override
@@ -60,6 +56,11 @@ public TriggerResult onEventTime(long time, W window, TriggerContext ctx) {
 		return TriggerResult.CONTINUE;
 	}
 
+	@Override
+	public void onFire(W window, TriggerContext ctx) throws Exception {
+		ctx.getPartitionedState(stateDesc).clear();
+	}
+
 	@Override
 	public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception {
 		return TriggerResult.CONTINUE;
@@ -79,10 +80,7 @@ public boolean canMerge() {
 	public TriggerResult onMerge(W window, OnMergeContext ctx) throws Exception {
 		ctx.mergePartitionedState(stateDesc);
 		ReducingState<Long> count = ctx.getPartitionedState(stateDesc);
-		if (count.get() >= maxCount) {
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
+		return (count.get() >= maxCount) ? TriggerResult.FIRE : TriggerResult.CONTINUE;
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
index 4a7262a0180..d5330b4108e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
@@ -46,8 +46,7 @@
 	private DeltaTrigger(double threshold, DeltaFunction<T> deltaFunction, TypeSerializer<T> stateSerializer) {
 		this.deltaFunction = deltaFunction;
 		this.threshold = threshold;
-		stateDesc = new ValueStateDescriptor<>("last-element", stateSerializer, null);
-
+		this.stateDesc = new ValueStateDescriptor<>("last-element", stateSerializer, null);
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
index da14ffd6461..ec8df3ba54d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
@@ -67,8 +67,7 @@ public boolean canMerge() {
 	}
 
 	@Override
-	public TriggerResult onMerge(TimeWindow window,
-			OnMergeContext ctx) {
+	public TriggerResult onMerge(TimeWindow window, OnMergeContext ctx) {
 		ctx.registerEventTimeTimer(window.maxTimestamp());
 		return TriggerResult.CONTINUE;
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
index f02d1db36d1..1e243e2ecf3 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
@@ -53,6 +53,11 @@ public TriggerResult onEventTime(long time, W window, TriggerContext ctx) throws
 		return triggerResult.isFire() ? TriggerResult.FIRE_AND_PURGE : triggerResult;
 	}
 
+	@Override
+	public void onFire(W window, TriggerContext ctx) throws Exception {
+		this.nestedTrigger.onFire(window, ctx);
+	}
+
 	@Override
 	public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception {
 		TriggerResult triggerResult = nestedTrigger.onProcessingTime(time, window, ctx);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
index ff80639479f..999c867efb7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
@@ -63,7 +63,8 @@
 	 * @param element The element that arrived.
 	 * @param timestamp The timestamp of the element that arrived.
 	 * @param window The window to which the element is being added.
-	 * @param ctx A context object that can be used to register timer callbacks.
+	 * @param ctx A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
 	 */
 	public abstract TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx) throws Exception;
 
@@ -72,7 +73,8 @@
 	 *
 	 * @param time The timestamp at which the timer fired.
 	 * @param window The window for which the timer fired.
-	 * @param ctx A context object that can be used to register timer callbacks.
+	 * @param ctx A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
 	 */
 	public abstract TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception;
 
@@ -81,10 +83,22 @@
 	 *
 	 * @param time The timestamp at which the timer fired.
 	 * @param window The window for which the timer fired.
-	 * @param ctx A context object that can be used to register timer callbacks.
+	 * @param ctx A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
 	 */
 	public abstract TriggerResult onEventTime(long time, W window, TriggerContext ctx) throws Exception;
 
+	/**
+	 * Called after a window has fired.
+	 *
+	 * @param window The window that fired.
+	 * @param ctx A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
+	 */
+	public void onFire(W window, TriggerContext ctx) throws Exception {
+
+	}
+
 	/**
 	 * Returns true if this trigger supports merging of trigger state and can therefore
 	 * be used with a
@@ -102,7 +116,8 @@ public boolean canMerge() {
 	 * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}.
 	 *
 	 * @param window The new window that results from the merge.
-	 * @param ctx A context object that can be used to register timer callbacks and access state.
+	 * @param ctx A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
 	 */
 	public TriggerResult onMerge(W window, OnMergeContext ctx) throws Exception {
 		throw new RuntimeException("This trigger does not support merging.");
@@ -154,7 +169,7 @@ public void clear(W window, TriggerContext ctx) throws Exception {}
 		 * @param time The time at which to invoke {@link Trigger#onProcessingTime(long, Window, TriggerContext)}
 		 */
 		void registerProcessingTimeTimer(long time);
-	
+
 		/**
 		 * Register an event-time callback. When the current watermark passes the specified
 		 * time {@link Trigger#onEventTime(long, Window, TriggerContext)} is called with the time specified here.
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/All.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/All.java
new file mode 100644
index 00000000000..dd241548818
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/All.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A {@link DslTrigger} that fires only if all its children propose to fire.
+ * @param <W> The type of {@link Window Windows} on which this {@code DslTrigger} can operate.
+ */
+public class All<W extends Window> extends DslTrigger<Object, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final List<DslTrigger<Object, W>> children;
+
+	private All(DslTrigger<Object, W>... triggers) {
+		Preconditions.checkNotNull(triggers, "The list of triggers must not be null");
+		Preconditions.checkNotNull(triggers.length == 0, "The list of triggers must not be empty");
+
+		this.children = Arrays.asList(triggers);
+		this.setIsRepeated(Repeated.ONCE);
+	}
+
+	@Override
+	List<DslTrigger<Object, W>> getChildTriggers() {
+		return children;
+	}
+
+	@Override
+	DslTrigger<Object, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		return this;
+	}
+
+	@Override
+	<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+		return Collections.emptyList();
+	}
+
+	@Override
+	boolean onElement(Object element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		boolean triggerResult = true;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerResult = triggerResult & triggerInvokable.invokeOnElement(element, timestamp, window);
+		}
+		return triggerResult;
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+		boolean triggerResult = true;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			boolean part = triggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+			triggerResult = triggerResult && part;
+		}
+		return triggerResult;
+	}
+
+	@Override
+	void onFire(W window, DslTriggerContext context) throws Exception {
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerInvokable.invokeOnFire(window);
+		}
+	}
+
+	@Override
+	public boolean canMerge() {
+		boolean canMerge = true;
+		for (DslTrigger trigger: this.children) {
+			canMerge = canMerge && trigger.canMerge();
+		}
+		return canMerge;
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		boolean triggerResult = true;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			boolean part = triggerInvokable.invokeOnMerge(window);
+			triggerResult = triggerResult && part;
+		}
+		return triggerResult;
+	}
+
+	public String toString() {
+		StringBuilder str = new StringBuilder();
+		str.append("All.of(");
+		Iterator<DslTrigger<Object, W>> it = children.iterator();
+		for (;;) {
+			DslTrigger<Object, W> trigger = it.next();
+			str.append(trigger);
+			if (!it.hasNext()) {
+				return str.append(")").toString();
+			}
+			str.append(", ");
+		}
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerInvokable.invokeClear(window);
+		}
+	}
+
+	public static <W extends Window> All<W> of(DslTrigger<Object, W>... triggers) {
+		return new All<>(triggers);
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Any.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Any.java
new file mode 100644
index 00000000000..7ac591ab0cd
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Any.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A {@link DslTrigger} that fires if at least one of its children propose to fire.
+ * @param <W> The type of {@link Window Windows} on which this {@code DslTrigger} can operate.
+ */
+public class Any<W extends Window> extends DslTrigger<Object, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final List<DslTrigger<Object, W>> children;
+
+	private Any(DslTrigger<Object, W>... triggers) {
+		Preconditions.checkNotNull(triggers, "The list of triggers must not be null");
+		Preconditions.checkNotNull(triggers.length == 0, "The list of triggers must not be empty");
+
+		this.children = Arrays.asList(triggers);
+		this.setIsRepeated(Repeated.ONCE);
+	}
+
+	@Override
+	List<DslTrigger<Object, W>> getChildTriggers() {
+		return children;
+	}
+
+	@Override
+	DslTrigger<Object, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		return this;
+	}
+
+	@Override
+	<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+		return Collections.emptyList();
+	}
+
+	@Override
+	boolean onElement(Object element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		boolean triggerResult = false;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			boolean part = triggerInvokable.invokeOnElement(element, timestamp, window);
+			triggerResult = triggerResult || part;
+		}
+		return triggerResult;
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+		boolean shouldFire = false;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			boolean part = triggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+			shouldFire = shouldFire || part;
+		}
+		return shouldFire;
+	}
+
+	@Override
+	void onFire(W window, DslTriggerContext context) throws Exception {
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerInvokable.invokeOnFire(window);
+		}
+	}
+
+	@Override
+	public boolean canMerge() {
+		boolean canMerge = true;
+		for (DslTrigger trigger: this.children) {
+			canMerge = canMerge && trigger.canMerge();
+		}
+		return canMerge;
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		boolean triggerResult = false;
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerResult = triggerResult || triggerInvokable.invokeOnMerge(window);
+		}
+		return triggerResult;
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder str = new StringBuilder();
+		str.append("Any.of(");
+		Iterator<DslTrigger<Object, W>> it = children.iterator();
+		for (;;) {
+			DslTrigger<Object, W> trigger = it.next();
+			str.append(trigger);
+			if (!it.hasNext()) {
+				return str.append(")").toString();
+			}
+			str.append(", ");
+		}
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		for (int childIdx = 0; childIdx < children.size(); childIdx++) {
+			DslTriggerInvokable<Object, W> triggerInvokable = context.getChildInvokable(childIdx);
+			triggerInvokable.invokeClear(window);
+		}
+	}
+
+	public static <W extends Window> Any<W> of(DslTrigger<Object, W>... triggers) {
+		return new Any<>(triggers);
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Count.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Count.java
new file mode 100644
index 00000000000..9b17c4a238c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Count.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DslTrigger} that fires once the count of elements in a pane reaches or exceeds the given count.
+ * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+ */
+@PublicEvolving
+public class Count<W extends Window> extends DslTrigger<Object, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final long maxCount;
+
+	private ReducingStateDescriptor<Long> stateDesc;
+
+	private Count(long maxCount) {
+		this.maxCount = maxCount;
+		this.setIsRepeated(Repeated.ONCE);
+	}
+
+	@Override
+	List<DslTrigger<Object, W>> getChildTriggers() {
+		return new ArrayList<>();
+	}
+
+	@Override
+	DslTrigger<Object, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		String descriptorName = "count_" + maxCount;
+		this.stateDesc = new ReducingStateDescriptor<>(descriptorName, new Sum(), LongSerializer.INSTANCE);
+		return this;
+	}
+
+	@Override
+	List<ReducingStateDescriptor<Long>> getStateDescriptors() {
+		if (stateDesc == null) {
+			throw new IllegalStateException("The state descriptor has not been initialized. " +
+				"Please call translate() before asking for state descriptors.");
+		}
+		List<ReducingStateDescriptor<Long>> descriptors = new ArrayList<>();
+		descriptors.add(this.stateDesc);
+		return descriptors;
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+
+		// the count.get() != null is for cases where the state has not been initialized,
+		// e.g. a count late trigger on a window with no late elements. In this case, the
+		// cleanup timer will fire and we will have an NPE here if we do not check.
+		ReducingState<Long> count = context.getPartitionedState(stateDesc);
+		Long counter = count.get();
+		return counter != null && counter >= maxCount;
+	}
+
+	@Override
+	boolean onElement(Object element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		ReducingState<Long> count = context.getPartitionedState(stateDesc);
+		count.add(1L);
+		return count.get() >= maxCount;
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		context.getPartitionedState(stateDesc).clear();
+	}
+
+	@Override
+	void onFire(Window window, DslTriggerContext context) throws Exception {
+		context.getPartitionedState(stateDesc).clear();
+	}
+
+	@Override
+	public boolean canMerge() {
+		return true;
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		context.mergePartitionedState(stateDesc);
+		ReducingState<Long> countState = context.getPartitionedState(stateDesc);
+		Long count = countState.get();
+		return (count != null && count >= maxCount);
+	}
+
+	@Override
+	public String toString() {
+		return "Count.atLeast(" +  maxCount + ")";
+	}
+
+	/**
+	 * Creates a trigger that fires once the number of elements in a pane reaches the given count.
+	 *
+	 * @param maxCount The count of elements at which to fire.
+	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
+	 */
+	public static <W extends Window> Count<W> atLeast(long maxCount) {
+		return new Count<>(maxCount);
+	}
+
+	private static class Sum implements ReduceFunction<Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long reduce(Long value1, Long value2) throws Exception {
+			return value1 + value2;
+		}
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTrigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTrigger.java
new file mode 100644
index 00000000000..38a3c8a54b6
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTrigger.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+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.typeutils.TypeSerializer;
+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 java.io.Serializable;
+import java.util.List;
+
+/**
+ * The specification of a dsl trigger.
+ *
+ * @param <T> The type of elements on which this {@code Trigger} works.
+ * @param <W> The type of {@link Window Windows} on which this {@code Trigger} can operate.
+ */
+public abstract class DslTrigger<T, W extends Window> implements Serializable {
+
+	private static final long serialVersionUID = -4104633972991191369L;
+
+	/**
+	 * Flag indicating if the trigger purges the window state after every firing
+	 * (<tt>discarding</tt>), or it only purges it at clean up time (<tt>accumulating</tt>).
+	 * By default triggers operate in <tt>accumulating</tt> mode.
+	 */
+	private boolean isDiscarding = false;
+
+	/**
+	 * Flag indicating if the trigger should fire throughout the lifespan of
+	 * a window, i.e. <tt>[start ... end + allowedLateness]</tt>, or fire once and
+	 * then never again.
+	 */
+	private Repeated isRepeated = Repeated.UNDEFINED;
+
+	/**
+	 * Makes the trigger operate in <tt>discarding</tt> mode, i.e.
+	 * purge the window state after every firing. By default dsl triggers
+	 * operate in <tt>accumulating</tt> mode, i.e. they do not purge the
+	 * window state after each firing, but only at clean up time.
+	 */
+	public DslTrigger<T, W> discarding() {
+		setIsDiscarding(true);
+		return this;
+	}
+
+	/**
+	 * Makes the trigger operate in <tt>accumulating</tt> mode, i.e.
+	 * it does not purge the window state after every firing, but only
+	 * at clean up time. This is the default mode.
+	 */
+	public DslTrigger<T, W> accumulating() {
+		setIsDiscarding(false);
+		return this;
+	}
+
+	boolean isDiscarding() {
+		return this.isDiscarding;
+	}
+
+	Repeated isRepeated() {
+		return this.isRepeated;
+	}
+
+	void setIsDiscarding(boolean isDiscarding) {
+		this.isDiscarding = isDiscarding;
+	}
+
+	void setIsRepeated(Repeated isRepeated) {
+		this.isRepeated = isRepeated;
+	}
+
+	/**
+	 * As described in the {@link DslTriggerRunner} dsl triggers can be composed in tree-like structures.
+	 * This method returns the children of this trigger in the tree.
+	 *
+	 * @return The list of child triggers.
+	 */
+	abstract List<DslTrigger<T, W>> getChildTriggers();
+
+	/**
+	 * Takes the specification of the {@link DslTrigger} and translates into into the most suitable implementation.
+	 *
+	 * @param windowSerializer the serializer for the windows created by the {@link WindowAssigner}.
+	 * @param allowedLateness the allowed lateness as specified by the user.
+	 * @return The implementation to be used.
+	 */
+	abstract DslTrigger<T, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness);
+
+	abstract <S extends State> List<StateDescriptor<S, ?>> getStateDescriptors();
+
+	/**
+	 * Determines if the trigger should fire because of a given timer.
+	 *
+	 * @param time The timestamp of the timer.
+	 * @param isEventTimeTimer <tt>true</tt> if it is an event time timer, <tt>false</tt> if it is a processing time one.
+	 * @param context A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
+	 */
+	abstract boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception;
+
+	/**
+	 * Called for every element that gets added to a pane. The result of this will determine
+	 * whether the pane is evaluated to emit results.
+	 *
+	 * @param element The element that arrived.
+	 * @param timestamp The timestamp of the element that arrived.
+	 * @param window The window to which the element is being added.
+	 * @param context A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
+	 */
+	abstract boolean onElement(T element, long timestamp, W window, DslTriggerContext context) throws Exception;
+
+	/**
+	 * Called after a window has fired.
+	 *
+	 * @param window The window that fired.
+	 * @param context A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
+	 */
+	abstract void onFire(W window, DslTriggerContext context) throws Exception;
+
+	/**
+	 * Returns true if this trigger supports merging of trigger state and can therefore
+	 * be used with a
+	 * {@link org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner}.
+	 *
+	 * <p>If this returns {@code true} you must properly implement
+	 * {@link #onMerge(Window, DslTriggerContext)}
+	 */
+	public boolean canMerge() {
+		return false;
+	}
+
+	/**
+	 * Called when several windows have been merged into one window by the
+	 * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}.
+	 *
+	 * @param window The new window that results from the merge.
+	 * @param context A context object that can be used to register timer callbacks, check the
+	 *            current event and processing time and get access to registered state.
+	 */
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		throw new RuntimeException("This trigger does not support merging.");
+	}
+
+	/**
+	 * Clears any state that the trigger might still hold for the given window.
+	 *
+	 * <p>By default, this method does nothing.
+	 */
+	void clear(W window, DslTriggerContext context) throws Exception {}
+
+	/**
+	 * A context allowing {@link DslTrigger dsl triggers} to register timers, query the time
+	 * (event and processing) and fetch trigger-related state from the state backend.
+	 */
+	interface DslTriggerContext extends Serializable {
+
+		/**
+		 * Sets the current invokable and the trigger context.
+		 */
+		void setTtriggerContext(Trigger.TriggerContext ctx, DslTriggerInvokable invokable);
+
+		/**
+		 * Returns the invokable corresponding to the child trigger at position <tt>index</tt> of the
+		 * current trigger. The current is assumed to have been already set by the
+		 * {@link #setTtriggerContext(Trigger.TriggerContext, DslTriggerInvokable)} method.
+		 */
+		<T, W extends Window> DslTriggerInvokable<T, W> getChildInvokable(int index);
+
+		/**
+		 * Returns the current processing time.
+		 */
+		long getCurrentProcessingTime();
+
+		/**
+		 * Returns the current watermark time.
+		 */
+		long getCurrentWatermark();
+
+		/**
+		 * Register a system time callback. When the current system time passes the specified
+		 * time {@link Trigger#onProcessingTime(long, Window, Trigger.TriggerContext)} is called with the time specified here.
+		 *
+		 * @param time The time at which to invoke {@link Trigger#onProcessingTime(long, Window, Trigger.TriggerContext)}
+		 */
+		void registerProcessingTimeTimer(long time);
+
+		/**
+		 * Register an event-time callback. When the current watermark passes the specified
+		 * time {@link Trigger#onEventTime(long, Window, Trigger.TriggerContext)} is called with the time specified here.
+		 *
+		 * @param time The watermark at which to invoke {@link Trigger#onEventTime(long, Window, Trigger.TriggerContext)}
+		 * @see org.apache.flink.streaming.api.watermark.Watermark
+		 */
+		void registerEventTimeTimer(long time);
+
+		@SuppressWarnings("unchecked")
+		<S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor);
+
+		@SuppressWarnings("unchecked")
+		<S extends MergingState<?, ?>> void mergePartitionedState(StateDescriptor<S, ?> stateDescriptor);
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerInvokable.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerInvokable.java
new file mode 100644
index 00000000000..71d0c9bdd96
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerInvokable.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+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.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+class DslTriggerInvokable<T, W extends Window> implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final int triggerId;
+	private final DslTriggerInvokable<T, W> parent;
+	private final List<DslTriggerInvokable<T, W>> children = new ArrayList<>();
+
+	private final DslTrigger<T, W> trigger;
+	private final DslTriggerContext context;
+
+	private final Map<String, StateDescriptor<? extends State, ?>> stateDescriptorMap = new HashMap<>();
+	private final Repeated isRepeated;
+
+	DslTriggerInvokable(int id, DslTriggerInvokable<T, W> parent, DslTrigger<T, W> trigger) {
+		Preconditions.checkNotNull(trigger);
+
+		this.triggerId = id;
+		this.parent = parent;
+		this.trigger = trigger;
+		this.context = new DslTriggerContext();
+		this.isRepeated = this.trigger.isRepeated();
+	}
+
+	void setContext(Trigger.TriggerContext context) {
+		this.context.setTtriggerContext(context, this);
+	}
+
+	/**
+	 * This method traverses the tree of {@link DslTriggerInvokable invokables} towards the root
+	 * until it finds the first that has a defined repeated flag. As soon as this is met, then
+	 * the rest of the subtree tree has the same behavior.
+	 */
+	Repeated isRepeated() {
+		return (parent == null || !isRepeated.isUndefined()) ? isRepeated : parent.isRepeated();
+	}
+
+	private DslTriggerInvokable<T, W> getChildInvokable(int index) {
+		if (index >= children.size()) {
+			throw new RuntimeException("Requested child at position " + index +
+				" but trigger " + trigger + " has " + children.size() +" children.");
+		}
+		return this.children.isEmpty() ? null : this.children.get(index);
+	}
+
+	private <S extends State> StateDescriptor<S, ?> getStateDescriptor(StateDescriptor<S, ?> stateDescriptor) {
+		StateDescriptor<S, ?> descriptor = (StateDescriptor<S, ?>) stateDescriptorMap.get(stateDescriptor.getName());
+		if (descriptor == null) {
+			throw new RuntimeException("Unknown state descriptor: " + stateDescriptor.getName());
+		}
+		return descriptor;
+	}
+
+	void addChild(DslTriggerInvokable<T, W> child) {
+		this.children.add(child);
+	}
+
+	boolean invokeShouldFire(long time, boolean isEventTimeTimer, W window) throws Exception {
+		return this.trigger.shouldFireOnTimer(time, isEventTimeTimer, window, context);
+	}
+
+	boolean invokeOnElement(T element, long timestamp, W window) throws Exception {
+		return this.trigger.onElement(element, timestamp, window, context);
+	}
+
+	void invokeOnFire(W window) throws Exception {
+		this.trigger.onFire(window, context);
+	}
+
+	boolean invokeOnMerge(W window) throws Exception {
+		return this.trigger.onMerge(window, context);
+	}
+
+	void invokeClear(W window) throws Exception {
+		this.trigger.clear(window, context);
+	}
+
+	<S extends State> StateDescriptor<S, ?> translateStateDescriptor(StateDescriptor<S, ?> stateDescriptor) {
+		String descName = stateDescriptor.getName();
+		StateDescriptor<S, ?> newStateDescriptor = (StateDescriptor<S, ?>) stateDescriptorMap.get(descName);
+		if (newStateDescriptor == null) {
+			newStateDescriptor = getStateDescriptorWithId(stateDescriptor);
+			stateDescriptorMap.put(descName, newStateDescriptor);
+		}
+		return newStateDescriptor;
+	}
+
+	private <S extends State> StateDescriptor<S, ?> getStateDescriptorWithId(StateDescriptor<S, ?> stateDescriptor) {
+		return StateDescriptor.translateStateDescriptorWithSuffix(stateDescriptor, "id=" + this.triggerId + ".trigger");
+	}
+
+	@Override
+	public String toString() {
+		return toStringWithPrefix("");
+	}
+
+	private String toStringWithPrefix(String prefix) {
+		StringBuilder str = new StringBuilder();
+		str.append(prefix).append(this.triggerId).append(" : ").append(this.trigger.toString());
+		if (this.children != null) {
+			for (DslTriggerInvokable invokable: this.children) {
+				str.append("\n").append(invokable.toStringWithPrefix(prefix + "\t"));
+			}
+		}
+		return str.toString();
+	}
+
+	private class DslTriggerContext implements DslTrigger.DslTriggerContext {
+
+		private transient Trigger.TriggerContext ctx;
+
+		@Override
+		public void setTtriggerContext(Trigger.TriggerContext ctx, DslTriggerInvokable invokable) {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			this.ctx = ctx;
+		}
+
+		@Override
+		public DslTriggerInvokable getChildInvokable(int index) {
+			DslTriggerInvokable child = DslTriggerInvokable.this.getChildInvokable(index);
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			child.setContext(ctx);
+			return child;
+		}
+
+		@Override
+		public long getCurrentProcessingTime() {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			return ctx.getCurrentProcessingTime();
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			return ctx.getCurrentWatermark();
+		}
+
+		@Override
+		public void registerProcessingTimeTimer(long time) {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			ctx.registerProcessingTimeTimer(time);
+		}
+
+		@Override
+		public void registerEventTimeTimer(long time) {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			ctx.registerEventTimeTimer(time);
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public <S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor) {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			}
+			try {
+				StateDescriptor<S, ?> actualStateDescriptor = getStateDescriptor(stateDescriptor);
+				return ctx.getPartitionedState(actualStateDescriptor);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state.", e);
+			}
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public <S extends MergingState<?, ?>> void mergePartitionedState(StateDescriptor<S, ?> stateDescriptor) {
+			if (ctx == null) {
+				throw new RuntimeException("The TriggerContext has not been properly initialized.");
+			} else if (!(ctx instanceof Trigger.OnMergeContext)) {
+				throw new RuntimeException("The TriggerContext does not support merging.");
+			}
+
+			StateDescriptor<S, ?> actualStateDescriptor = getStateDescriptor(stateDescriptor);
+			((Trigger.OnMergeContext) ctx).mergePartitionedState(actualStateDescriptor);
+		}
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerRunner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerRunner.java
new file mode 100644
index 00000000000..c0046f5d1fe
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/DslTriggerRunner.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+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.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link Trigger} used to run the user specified {@link DslTrigger}. Dsl triggers
+ * are allowed to be composed into tree-like structures, e.g.:
+ *
+ * <pre>{@code
+ *
+ *         EventTime.<TimeWindow>afterEndOfWindow()
+ *             .withEarlyTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100)))
+ *             .withLateTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(200)))
+ *             .discarding()
+ * }</pre>
+ *
+ * Given this specification, the runner will build a tree of {@link DslTriggerInvokable TriggerInvokables},
+ * each responsible for executing one of the dsl trigger specification in the tree. After this is done,
+ * the runner is used as a classic trigger and makes sure that the trigger calls are propagated correctly
+ * through the dsl trigger tree.
+ *
+ * @param <T> The type of elements on which this {@code Trigger} works.
+ * @param <W> The type of {@link Window Windows} on which this {@code Trigger} can operate.
+ */
+public class DslTriggerRunner<T, W extends Window> extends Trigger<T, W> {
+
+	/** The root dsl trigger. */
+	private final DslTrigger<T, W> trigger;
+
+	/** The invokable corresponding to the root dsl trigger. */
+	private DslTriggerInvokable<T, W> invokable;
+
+	public DslTriggerRunner(DslTrigger<T, W> trigger) {
+		Preconditions.checkNotNull(trigger, "The trigger cannot be null");
+		this.trigger = trigger.isRepeated().once() ? Repeat.Once(trigger) : trigger;
+		this.trigger.setIsDiscarding(trigger.isDiscarding());
+	}
+
+	public Trigger<T, W> getTrigger() {
+		return this;
+	}
+
+	/**
+	 * Creates the tree of {@link DslTriggerInvokable TriggerInvokables} corresponding to the
+	 * dsl trigger specification. In addition, it makes sure that the specified trigger
+	 * combinations are valid. This method is called by the
+	 * {@link org.apache.flink.streaming.runtime.operators.windowing.WindowOperator}.
+	 * @param windowSerializer The serializer of the windows used.
+	 * @param allowedLateness The specified allowed lateness.
+	 */
+	public void createTriggerTree(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		Map<Integer, DslTriggerInvokable<T, W>> triggerIdToInvokableMapping = new HashMap<>();
+		this.createTriggerInvokableTree(trigger, null, new ArrayList<DslTrigger<T, W>>(),
+			triggerIdToInvokableMapping, windowSerializer, allowedLateness);
+		this.invokable = triggerIdToInvokableMapping.get(0);
+	}
+
+	/**
+	 * Recursively creates a {@link DslTriggerInvokable} for each {@link Trigger trigger} in the
+	 * (possibly composite) trigger used by the window operator and creates a mapping
+	 * between the created invokable and its unique id in the trigger tree.
+	 */
+	private void createTriggerInvokableTree(DslTrigger<T, W> trigger,
+											DslTriggerInvokable<T, W> parentInvokable,
+											List<DslTrigger<T, W>> flattenedTriggerList,
+											Map<Integer, DslTriggerInvokable<T, W>> triggerIdToInvokableMapping,
+											TypeSerializer<W> windowSerializer,
+											long allowedLateness) {
+
+		DslTrigger<T, W> actualTrigger = trigger.translate(windowSerializer, allowedLateness);
+		flattenedTriggerList.add(actualTrigger);
+		int actualTriggerId = flattenedTriggerList.lastIndexOf(actualTrigger);
+
+		DslTriggerInvokable<T, W> actualTriggerInvokable = createInvokable(actualTriggerId, parentInvokable, actualTrigger);
+		triggerIdToInvokableMapping.put(actualTriggerId, actualTriggerInvokable);
+
+		List<DslTrigger<T, W>> childTriggers = actualTrigger.getChildTriggers();
+		if (childTriggers == null || childTriggers.isEmpty()) {
+			return;
+		}
+
+		Repeated parentIsRepeated = actualTriggerInvokable.isRepeated();
+		for(DslTrigger<T, W> childTrigger : childTriggers) {
+
+			// set the isRepeated flag
+			if (parentIsRepeated.isUndefined() && childTrigger.isRepeated().once()) {
+				// we cannot explicitly set the ONCE, so if the repeated flag of the parent is UNDEFINED,
+				// e.g. in the after-end-of-window triggers, and we are the first ONCE in the tree, then
+				// state it explicitly by wrapping the trigger in a Repeat.Once().
+				childTrigger = Repeat.Once(childTrigger);
+			}
+
+			createTriggerInvokableTree(childTrigger, actualTriggerInvokable, flattenedTriggerList,
+				triggerIdToInvokableMapping, windowSerializer, allowedLateness);
+
+			// this works because we cannot have in the same branch of the trigger tree the same
+			// object (dslTrigger) twice. There is no way to say "put myself in my subtree".
+
+			int childId = flattenedTriggerList.lastIndexOf(childTrigger);
+			DslTriggerInvokable<T, W> childInvokable = triggerIdToInvokableMapping.get(childId);
+			actualTriggerInvokable.addChild(childInvokable);
+		}
+	}
+
+	private DslTriggerInvokable<T, W> createInvokable(int id, DslTriggerInvokable<T, W> parent, DslTrigger<T, W> child) {
+		DslTriggerInvokable<T, W> invokable = new DslTriggerInvokable<>(id, parent, child);
+
+		List<StateDescriptor<State, ?>> descriptors = child.getStateDescriptors();
+		if (descriptors != null) {
+			for (StateDescriptor<State, ?> descriptor: descriptors) {
+				invokable.translateStateDescriptor(descriptor);
+			}
+		}
+		return invokable;
+	}
+
+	@Override
+	public TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx) throws Exception {
+		invokable.setContext(ctx);
+		boolean triggerResult = invokable.invokeOnElement(element, timestamp, window);
+		if (!triggerResult) {
+			return TriggerResult.CONTINUE;
+		}
+		return trigger.isDiscarding() ? TriggerResult.FIRE_AND_PURGE : TriggerResult.FIRE;
+	}
+
+	@Override
+	public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception {
+		return shouldFireOnTimer(time, window, ctx, false);
+	}
+
+	@Override
+	public TriggerResult onEventTime(long time, W window, TriggerContext ctx) throws Exception {
+		return shouldFireOnTimer(time, window, ctx, true);
+	}
+
+	private TriggerResult shouldFireOnTimer(long time, W window, TriggerContext ctx, boolean isEventTime) throws Exception {
+		invokable.setContext(ctx);
+		boolean triggerResult = invokable.invokeShouldFire(time, isEventTime, window);
+		if (!triggerResult) {
+			return TriggerResult.CONTINUE;
+		}
+		return trigger.isDiscarding() ? TriggerResult.FIRE_AND_PURGE : TriggerResult.FIRE;
+	}
+
+	@Override
+	public void onFire(W window, TriggerContext ctx) throws Exception {
+		invokable.setContext(ctx);
+		invokable.invokeOnFire(window);
+	}
+
+	@Override
+	public void clear(W window, TriggerContext ctx) throws Exception {
+		invokable.setContext(ctx);
+		invokable.invokeClear(window);
+	}
+
+	@Override
+	public boolean canMerge() {
+		return trigger.canMerge();
+	}
+
+	@Override
+	public TriggerResult onMerge(W window, OnMergeContext ctx) throws Exception {
+		invokable.setContext(ctx);
+		boolean triggerResult = invokable.invokeOnMerge(window);
+		if (!triggerResult) {
+			return TriggerResult.CONTINUE;
+		}
+		return trigger.isDiscarding() ? TriggerResult.FIRE_AND_PURGE : TriggerResult.FIRE;
+	}
+
+	@Override
+	public String toString() {
+		return trigger.toString() + ((trigger.isDiscarding() ? ".discarding()" : ".accumulating()"));
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/EventTime.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/EventTime.java
new file mode 100644
index 00000000000..050f4522b3c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/EventTime.java
@@ -0,0 +1,588 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+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.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@link DslTrigger} that reacts to event-time timers.
+ * The behavior can be one of the following:
+ * <p><ul>
+ *      <li/> fire when the watermark passes the end of the window ({@link EventTime#afterEndOfWindow()}),
+ *      <li/> fire when the event time advances by a certain interval
+ *            after reception of the first element after the last firing for
+ *            a given window ({@link EventTime#afterFirstElement(Time)}).
+ * </ul></p>
+ * In the first case, the trigger can also specify an <tt>early</tt> and a <tt>late</tt> trigger.
+ * The <tt>early trigger</tt> will be responsible for specifying when the trigger should fire in the period
+ * between the beginning of the window and the time when the watermark passes the end of the window.
+ * The <tt>late trigger</tt> takes over after the watermark passes the end of the window, and specifies when
+ * the trigger should fire in the period between the <tt>endOfWindow</tt> and <tt>endOfWindow + allowedLateness</tt>.
+ *
+ * @param <W> The type of {@link Window Windows} on which this {@code DslTrigger} can operate.
+ */
+public class EventTime<W extends Window> extends DslTrigger<Object, W> {
+	private static final long serialVersionUID = 1L;
+
+	private static final long UNDEFINED_INTERVAL = -1;
+
+	private final long interval;
+
+	private DslTrigger<Object, W> earlyTrigger;
+
+	private DslTrigger<Object, W> lateTrigger;
+
+	private DslTrigger<Object, W> actualTrigger;
+
+	private EventTime(long interval) {
+		Preconditions.checkArgument(interval >= 0 || interval == UNDEFINED_INTERVAL);
+		this.interval = interval;
+		if (this.interval == UNDEFINED_INTERVAL) {
+			this.setIsRepeated(Repeated.UNDEFINED);
+		} else {
+			this.setIsRepeated(Repeated.ONCE);
+		}
+	}
+
+	private boolean isAfterEndOfWindow() {
+		return interval == UNDEFINED_INTERVAL;
+	}
+
+	/**
+	 * In the case of an {@link #afterEndOfWindow()} trigger, this method allows the specification of an <tt>early</tt> trigger.
+	 * This trigger will be responsible for specifying when the trigger should fire in the period between the beginning
+	 * of the window and the time when the watermark passes the end of the window. If no early trigger is specified, then
+	 * no firing happens within this period.
+	 *
+	 * @param earlyFiringTrigger The specification of the early trigger.
+	 */
+	public EventTime<W> withEarlyTrigger(DslTrigger<Object, W> earlyFiringTrigger) {
+		this.earlyTrigger = earlyFiringTrigger;
+		return this;
+	}
+
+	/**
+	 * In the case of an {@link #afterEndOfWindow()} trigger, this method allows the specification of an <tt>late</tt> trigger.
+	 * This trigger takes over after the watermark passes the end of the window, and specifies when the trigger should fire
+	 * in the period between the <tt>endOfWindow</tt> and <tt>endOfWindow + allowedLateness</tt>. If no late trigger is specified,
+	 * then we have a firing for every late element that arrives.
+	 *
+	 * @param lateFiringTrigger The specification of the late trigger.
+	 * */
+	public EventTime<W> withLateTrigger(DslTrigger<Object, W> lateFiringTrigger) {
+		this.lateTrigger = lateFiringTrigger;
+		return this;
+	}
+
+	@Override
+	List<DslTrigger<Object, W>> getChildTriggers() {
+		return this.actualTrigger.getChildTriggers();
+	}
+
+	@Override
+	DslTrigger<Object, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		if (isAfterEndOfWindow() && windowSerializer instanceof GlobalWindow.Serializer) {
+			throw new IllegalStateException("An EventTimeTrigger.afterEndOfWindow() will never fire with GlobalWindows.");
+		}
+
+		if (earlyTrigger != null && !isAfterEndOfWindow()) {
+			throw new UnsupportedOperationException("An EventTimeTrigger.afterFirstElement() cannot have early firings.");
+		}
+
+		if (lateTrigger != null && !isAfterEndOfWindow()) {
+			throw new UnsupportedOperationException("An EventTimeTrigger.afterFirstElement() cannot have late firings.");
+		}
+
+		if (isAfterEndOfWindow()) {
+			this.actualTrigger = lateTrigger == null || allowedLateness == 0 ?
+				new AfterEndOfWindowNoState<>(earlyTrigger) :
+				new AfterEndOfWindow<>(earlyTrigger, lateTrigger);
+		} else {
+			this.actualTrigger = new AfterFirstElementInPane<>(interval);
+		}
+		this.actualTrigger = actualTrigger.translate(windowSerializer, allowedLateness);
+		return this;
+	}
+
+	@Override
+	<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+		return this.actualTrigger.getStateDescriptors();
+	}
+
+	@Override
+	boolean onElement(Object element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.onElement(element, timestamp, window, context);
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.shouldFireOnTimer(time, isEventTimeTimer, window, context);
+	}
+
+	@Override
+	void onFire(W window, DslTriggerContext context) throws Exception {
+		this.actualTrigger.onFire(window, context);
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		this.actualTrigger.clear(window, context);
+	}
+
+	@Override
+	public boolean canMerge() {
+		return true;
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.onMerge(window, context);
+	}
+
+	@Override
+	public String toString() {
+		return !isAfterEndOfWindow() ?
+			"EventTimeTrigger.afterFirstElement(" + interval + " ms)" :
+			"EventTimeTrigger.afterEndOfWindow()" +
+				(earlyTrigger == null ? "" : ".withEarlyTrigger(" + earlyTrigger.toString() + ")") +
+				(lateTrigger == null ? "" : ".withLateTrigger(" + lateTrigger.toString() + ")");
+	}
+
+	/**
+	 * Creates a trigger that fires when the watermark passes the end of the window.
+	 * This trigger allows the additional specification of an early (see {@link #withEarlyTrigger(DslTrigger)})
+	 * and/or a late trigger (see {@link #withLateTrigger(DslTrigger)}).
+	 */
+	public static <W extends Window> EventTime<W> afterEndOfWindow() {
+		return new EventTime<>(UNDEFINED_INTERVAL);
+	}
+
+	/**
+	 * Creates a trigger that fires when the event time advances by a certain <tt>interval</tt> after reception of
+	 * the first element after the last firing of a given window.
+	 * @param interval The interval by which the event time should advance.
+	 */
+	public static <W extends Window> EventTime<W> afterFirstElement(Time interval) {
+		return new EventTime<>(interval.toMilliseconds());
+	}
+
+	public static <W extends Window> EventTime<W> Default() {
+		return EventTime.afterEndOfWindow();
+	}
+
+	/**
+	 * Upon merging it returns the minimum among the values that are different than -1.
+	 */
+	private static class MinAmongSet implements ReduceFunction<Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long reduce(Long value1, Long value2) throws Exception {
+			return (value1 == -1L || value2 == -1L) ?
+				value1 + value2 + 1L :
+				Math.min(value1, value2);
+		}
+	}
+
+	private class AfterFirstElementInPane<P extends Window> extends DslTrigger<Object, P> {
+
+		private final long interval;
+
+		private ReducingStateDescriptor<Long> afterFirstElementStateDesc;
+
+		AfterFirstElementInPane(long interval) {
+			Preconditions.checkArgument(interval >= 0 || interval == UNDEFINED_INTERVAL);
+			this.interval = interval;
+		}
+
+		@Override
+		List<DslTrigger<Object, P>> getChildTriggers() {
+			return new ArrayList<>();
+		}
+
+		@Override
+		DslTrigger<Object, P> translate(TypeSerializer<P> windowSerializer, long allowedLateness) {
+			String descriptorName = "eventTime-afterFirstElement_" + interval;
+
+			this.afterFirstElementStateDesc = new ReducingStateDescriptor<>(
+				descriptorName, new MinAmongSet(), LongSerializer.INSTANCE);
+			return this;
+		}
+
+		@Override
+		List<ReducingStateDescriptor<Long>> getStateDescriptors() {
+			if (afterFirstElementStateDesc == null) {
+				throw new IllegalStateException("The state descriptor has not been initialized. " +
+					"Please call translate() before asking for state descriptors.");
+			}
+			List<ReducingStateDescriptor<Long>> descriptors = new ArrayList<>();
+			descriptors.add(this.afterFirstElementStateDesc);
+			return descriptors;
+		}
+
+		@Override
+		boolean onElement(Object element, long timestamp, P window, DslTriggerContext context) throws Exception {
+			// this is the afterFirstElementInPane mode
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			Long timer = nextFiring.get();
+
+			if (timer == null) {
+				long nextTimer = context.getCurrentWatermark() + interval;
+				context.registerEventTimeTimer(nextTimer);
+				nextFiring.add(nextTimer);
+			} else {
+				// this is the case after a merge
+				context.registerEventTimeTimer(timer);
+			}
+			return false;
+		}
+
+		@Override
+		boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, P window, DslTriggerContext context) throws Exception {
+			if (!isEventTimeTimer) {
+				return false;
+			}
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			Long timer = nextFiring.get();
+			return timer != null && timer == time;
+		}
+
+		@Override
+		void onFire(P window, DslTriggerContext context) throws Exception {
+			context.getPartitionedState(afterFirstElementStateDesc).clear();
+		}
+
+		@Override
+		public boolean canMerge() {
+			return true;
+		}
+
+		@Override
+		boolean onMerge(P window, DslTriggerContext context) throws Exception {
+			// the onElement will take care of setting a new timer
+			// if we are in afterEndOfWindow mode, or the merging
+			// revealed no already set timers
+
+			context.mergePartitionedState(afterFirstElementStateDesc);
+			return false;
+		}
+
+		@Override
+		public String toString() {
+			return "EventTimeTrigger.afterFirstElement(" + interval + " ms)";
+		}
+
+		@Override
+		void clear(P window, DslTriggerContext context) throws Exception {
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			nextFiring.clear();
+		}
+	}
+
+	private class AfterEndOfWindow<P extends Window> extends DslTrigger<Object, P> {
+
+		private final DslTrigger<Object, P> earlyTrigger;
+
+		private final DslTrigger<Object, P> lateTrigger;
+
+		private ValueStateDescriptor<Boolean> hasFiredOnTimeStateDesc;
+
+		AfterEndOfWindow(DslTrigger<Object, P> earlyTrigger, DslTrigger<Object, P> lateTrigger) {
+			this.earlyTrigger = earlyTrigger;
+			this.lateTrigger = lateTrigger;
+		}
+
+		@Override
+		public List<DslTrigger<Object, P>> getChildTriggers() {
+			List<DslTrigger<Object, P>> triggers = new ArrayList<>();
+			if (earlyTrigger != null) {
+				triggers.add(earlyTrigger);
+			}
+
+			if (lateTrigger != null) {
+				triggers.add(lateTrigger);
+			}
+			return triggers;
+		}
+
+		private DslTriggerInvokable<Object, P> getEarlyTriggerInvokable(DslTriggerContext ctx) {
+			if (earlyTrigger == null) {
+				return null;
+			}
+			return ctx.getChildInvokable(0);
+		}
+
+		private DslTriggerInvokable<Object, P> getLateTriggerInvokable(DslTriggerContext ctx) {
+			if (lateTrigger == null) {
+				return null;
+			} else if (earlyTrigger == null) {
+				return ctx.getChildInvokable(0);
+			}
+			return ctx.getChildInvokable(1);
+		}
+
+
+		@Override
+		DslTrigger<Object, P> translate(TypeSerializer<P> windowSerializer, long allowedLateness) {
+			String descriptorName = "eventTime-afterEOW";
+			this.hasFiredOnTimeStateDesc = new ValueStateDescriptor<>(
+				descriptorName, BooleanSerializer.INSTANCE, false);
+			return this;
+		}
+
+		@Override
+		List<ValueStateDescriptor<Boolean>> getStateDescriptors() {
+			if (this.hasFiredOnTimeStateDesc == null) {
+				throw new IllegalStateException("The state descriptor has not been initialized. " +
+					"Please call translate() before asking for state descriptors.");
+			}
+			List<ValueStateDescriptor<Boolean>> descriptors = new ArrayList<>();
+			descriptors.add(this.hasFiredOnTimeStateDesc);
+			return descriptors;
+		}
+
+		@Override
+		boolean onElement(Object element, long timestamp, P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean earlyTriggerResult = earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnElement(element, timestamp, window);
+
+			DslTriggerInvokable<Object, P> lateTriggerInvokable = getLateTriggerInvokable(context);
+			boolean lateTriggerResult = lateTriggerInvokable != null &&
+				lateTriggerInvokable.invokeOnElement(element, timestamp, window);
+
+			Boolean hasFiredOnTime = context.getPartitionedState(hasFiredOnTimeStateDesc).value();
+			if (hasFiredOnTime) {
+				// this is to cover the case where we recover from a failure and the watermark
+				// is Long.MIN_VALUE but the window is already in the late phase.
+				return lateTriggerInvokable != null && lateTriggerResult;
+			} else {
+				if (window.maxTimestamp() <= context.getCurrentWatermark()) {
+					// we are in the late phase
+
+					// if there is no late trigger then we fire on every late element
+					// This also covers the case of recovery after a failure
+					// where the currentWatermark will be Long.MIN_VALUE
+					return true;
+				} else {
+					// we are in the early phase
+					context.registerEventTimeTimer(window.maxTimestamp());
+					return earlyTriggerResult;
+				}
+			}
+		}
+
+		@Override
+		boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean shouldEarlyFire = earlyTriggerInvokable != null && earlyTriggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+
+			DslTriggerInvokable<Object, P> lateTriggerInvokable = getLateTriggerInvokable(context);
+			boolean shouldLateFire = lateTriggerInvokable != null && lateTriggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+
+			Boolean hasFiredOnTime = context.getPartitionedState(hasFiredOnTimeStateDesc).value();
+			if (hasFiredOnTime) {
+				return shouldLateFire;
+			} else {
+				return (isEventTimeTimer && time == window.maxTimestamp()) || shouldEarlyFire;
+			}
+		}
+
+		@Override
+		void onFire(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeOnFire(window);
+			}
+
+			DslTriggerInvokable<Object, P> lateTriggerInvokable = getLateTriggerInvokable(context);
+			if (lateTriggerInvokable != null) {
+				lateTriggerInvokable.invokeOnFire(window);
+			}
+
+			if (context.getCurrentWatermark() >= window.maxTimestamp()) {
+				ValueState<Boolean> hasFiredState = context.getPartitionedState(hasFiredOnTimeStateDesc);
+				if (!hasFiredState.value()) {
+					if (lateTriggerInvokable != null) {
+						lateTriggerInvokable.invokeClear(window);
+					}
+					if (earlyTriggerInvokable != null) {
+						earlyTriggerInvokable.invokeClear(window);
+					}
+					hasFiredState.update(true);
+				}
+			}
+		}
+
+		@Override
+		public boolean canMerge() {
+			return (earlyTrigger == null || earlyTrigger.canMerge()) &&
+				(lateTrigger == null || lateTrigger.canMerge());
+		}
+
+		@Override
+		boolean onMerge(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean earlyTriggerResult = earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnMerge(window);
+
+			DslTriggerInvokable<Object, P> lateTriggerInvokable = getLateTriggerInvokable(context);
+			boolean lateTriggerResult = lateTriggerInvokable != null &&
+				lateTriggerInvokable.invokeOnMerge(window);
+
+			// we assume that the new merged window has not fired yet its on-time timer.
+			context.getPartitionedState(hasFiredOnTimeStateDesc).update(false);
+
+			// the onElement() will register the timer for the end of the new window.
+			return window.maxTimestamp() <= context.getCurrentWatermark() ?
+				lateTriggerResult : earlyTriggerResult;
+		}
+
+		@Override
+		public String toString() {
+			return "EventTimeTrigger.afterEndOfWindow()" +
+				(earlyTrigger == null ? "" : ".withEarlyTrigger("+ earlyTrigger.toString() +")") +
+				(lateTrigger == null ? "" : ".withLateTrigger("+ lateTrigger.toString() +")");
+		}
+
+		@Override
+		void clear(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeClear(window);
+			}
+
+			DslTriggerInvokable<Object, P> lateTriggerInvokable = getLateTriggerInvokable(context);
+			if (lateTriggerInvokable != null) {
+				lateTriggerInvokable.invokeClear(window);
+			}
+			context.getPartitionedState(hasFiredOnTimeStateDesc).clear();
+		}
+	}
+
+	private class AfterEndOfWindowNoState<P extends Window> extends DslTrigger<Object, P> {
+
+		private final DslTrigger<Object, P> earlyTrigger;
+
+		AfterEndOfWindowNoState(DslTrigger<Object, P> earlyTrigger) {
+			this.earlyTrigger = earlyTrigger;
+		}
+
+		@Override
+		List<DslTrigger<Object, P>> getChildTriggers() {
+			List<DslTrigger<Object, P>> triggers = new ArrayList<>();
+			if (earlyTrigger != null) {
+				triggers.add(earlyTrigger);
+			}
+			return triggers;
+		}
+
+		DslTriggerInvokable<Object, P> getEarlyTriggerInvokable(DslTriggerContext ctx) {
+			if (earlyTrigger == null) {
+				return null;
+			}
+			return ctx.getChildInvokable(0);
+		}
+
+		@Override
+		DslTrigger<Object, P> translate(TypeSerializer<P> windowSerializer, long allowedLateness) {
+			return this;
+		}
+
+		@Override
+		<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+			return Collections.emptyList();
+		}
+
+		@Override
+		boolean onElement(Object element, long timestamp, P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean earlyTriggerResult = earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnElement(element, timestamp, window);
+			if (window.maxTimestamp() <= context.getCurrentWatermark()) {
+				// the on-time firing
+				return true;
+			} else {
+				// this is an early element so register the timer and let the early trigger decide
+				context.registerEventTimeTimer(window.maxTimestamp());
+				return earlyTriggerResult;
+			}
+		}
+
+		@Override
+		boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean shouldEarlyFire = earlyTriggerInvokable != null && earlyTriggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+			return (isEventTimeTimer && time == window.maxTimestamp()) || shouldEarlyFire;
+		}
+
+		@Override
+		void onFire(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeOnFire(window);
+			}
+		}
+
+		@Override
+		public boolean canMerge() {
+			return (earlyTrigger == null || earlyTrigger.canMerge());
+		}
+
+		@Override
+		boolean onMerge(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean earlyTriggerResult = earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnMerge(window);
+
+			// the onElement() will register the timer for the end of the new window.
+			return window.maxTimestamp() > context.getCurrentWatermark() && earlyTriggerResult;
+		}
+
+		@Override
+		public String toString() {
+			return "EventTimeTrigger.afterEndOfWindow()" +
+				(earlyTrigger == null ? "" : ".withEarlyTrigger(" + earlyTrigger.toString() + ")");
+		}
+
+		@Override
+		void clear(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeClear(window);
+			}
+		}
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/ProcessingTime.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/ProcessingTime.java
new file mode 100644
index 00000000000..7054aac44a4
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/ProcessingTime.java
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@link DslTrigger} that reacts to processing-time timers.
+ * The behavior can be one of the following:
+ * <p><ul>
+ *     <li> fire when the processing time passes the end of the window ({@link ProcessingTime#afterEndOfWindow()}),
+ *     <li> fire when the processing time advances by a certain interval
+ *          after reception of the first element after the last firing for
+ *          a given window ({@link ProcessingTime#afterFirstElement(Time)}).
+ * </ul></p>
+ * In the first case, the trigger can also specify an <tt>early</tt> trigger.
+ * The <tt>early trigger</tt> will be responsible for specifying when the trigger should fire in the period
+ * between the beginning of the window and the time when the processing time passes the end of the window.
+ *
+ * @param <W> The type of {@link Window Windows} on which this {@code DslTrigger} can operate.
+ */
+public class ProcessingTime<W extends Window> extends DslTrigger<Object, W> {
+	private static final long serialVersionUID = 1L;
+
+	private static final long UNDEFINED_INTERVAL = -1;
+
+	private final long interval;
+
+	private DslTrigger<Object, W> earlyTrigger;
+
+	private DslTrigger<Object, W> actualTrigger;
+
+	private ProcessingTime(long interval) {
+		// we do not check it here but if the interval is bigger than the
+		// duration of the window + allowed lateness, then the trigger will not fire.
+		Preconditions.checkArgument(interval >= 0 || interval == UNDEFINED_INTERVAL);
+		this.interval = interval;
+
+		if (this.interval == UNDEFINED_INTERVAL) {
+			this.setIsRepeated(Repeated.UNDEFINED);
+		} else {
+			this.setIsRepeated(Repeated.ONCE);
+		}
+	}
+
+	private boolean isAfterEndOfWindow() {
+		return interval == UNDEFINED_INTERVAL;
+	}
+
+	/**
+	 * In the case of an {@link #afterEndOfWindow()} trigger, this method allows the specification of an <tt>early</tt> trigger.
+	 * This trigger will be responsible for specifying when the trigger should fire in the period between the beginning
+	 * of the window and the time when the processing time passes the end of the window. If no early trigger is specified, then
+	 * no firing happens within this period.
+	 *
+	 * @param earlyFiringTrigger The specification of the early trigger.
+	 */
+	public ProcessingTime<W> withEarlyTrigger(DslTrigger<Object, W> earlyFiringTrigger) {
+		this.earlyTrigger = earlyFiringTrigger;
+		return this;
+	}
+
+	@Override
+	List<DslTrigger<Object, W>> getChildTriggers() {
+		List<DslTrigger<Object, W>> triggers = new ArrayList<>();
+		if (earlyTrigger != null) {
+			triggers.add(earlyTrigger);
+		}
+		return triggers;
+	}
+
+	@Override
+	DslTrigger<Object, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		if (isAfterEndOfWindow() && windowSerializer instanceof GlobalWindow.Serializer) {
+			throw new IllegalStateException("A ProcessingTimeTrigger.afterEndOfWindow() will never fire with GlobalWindows.");
+		} else if (!isAfterEndOfWindow() && earlyTrigger != null) {
+			throw new UnsupportedOperationException("ProcessingTimeTrigger.afterFirstElement() cannot have early firings.");
+		}
+
+		this.actualTrigger = isAfterEndOfWindow() ?
+			new AfterEndOfWindow<>(earlyTrigger) :
+			new AfterFirstElementInPane<W>(interval);
+
+		this.actualTrigger = actualTrigger.translate(windowSerializer, allowedLateness);
+		return this;
+	}
+
+	@Override
+	<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+		return this.actualTrigger.getStateDescriptors();
+	}
+
+	@Override
+	boolean onElement(Object element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.onElement(element, timestamp, window, context);
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.shouldFireOnTimer(time, isEventTimeTimer, window, context);
+	}
+
+	@Override
+	void onFire(W window, DslTriggerContext context) throws Exception {
+		this.actualTrigger.onFire(window, context);
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		this.actualTrigger.clear(window, context);
+	}
+
+	@Override
+	public boolean canMerge() {
+		return this.actualTrigger.canMerge();
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		return this.actualTrigger.onMerge(window, context);
+	}
+
+	@Override
+	public String toString() {
+		return !isAfterEndOfWindow() ?
+			"ProcessingTimeTrigger.afterFirstElement(" + interval + " ms)" :
+			"ProcessingTimeTrigger.afterEndOfWindow()" +
+				(earlyTrigger == null ? "" : ".withEarlyTrigger(" + earlyTrigger.toString() + ")");
+	}
+
+	/**
+	 * Creates a trigger that fires when the processing time passes the end of the window.
+	 * This trigger allows the additional specification of an early (see {@link #withEarlyTrigger(DslTrigger)}).
+	 */
+	public static <W extends Window> ProcessingTime<W> afterEndOfWindow() {
+		return new ProcessingTime<>(UNDEFINED_INTERVAL);
+	}
+
+	/**
+	 * Creates a trigger that fires when the processing time advances by a certain <tt>interval</tt>
+	 * after reception of the first element after the last firing of a given window.
+	 * @param interval The interval by which the event time should advance.
+	 */
+	public static <W extends Window> ProcessingTime<W> afterFirstElement(Time interval) {
+		return new ProcessingTime<>(interval.toMilliseconds());
+	}
+
+	/**
+	 * Upon merging it returns the minimum among the values that are different than -1.
+	 */
+	private static class MinAmongSet implements ReduceFunction<Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long reduce(Long value1, Long value2) throws Exception {
+			return (value1 == -1L || value2 == -1L) ?
+				value1 + value2 + 1L :
+				Math.min(value1, value2);
+		}
+	}
+
+	private class AfterFirstElementInPane<P extends Window> extends DslTrigger<Object, P> {
+		private final long interval;
+
+		private ReducingStateDescriptor<Long> afterFirstElementStateDesc;
+
+		AfterFirstElementInPane(long interval) {
+			Preconditions.checkArgument(interval >= 0 || interval == UNDEFINED_INTERVAL);
+			this.interval = interval;
+		}
+
+		@Override
+		List<DslTrigger<Object, P>> getChildTriggers() {
+			return new ArrayList<>();
+		}
+
+		@Override
+		DslTrigger<Object, P> translate(TypeSerializer<P> windowSerializer, long allowedLateness) {
+			String descriptorName = "processingTime-afterFirstElement_" + interval;
+			this.afterFirstElementStateDesc = new ReducingStateDescriptor<>(
+				descriptorName, new MinAmongSet(), LongSerializer.INSTANCE);
+			return this;
+		}
+
+		@Override
+		List<ReducingStateDescriptor<Long>> getStateDescriptors() {
+			if (afterFirstElementStateDesc == null) {
+				throw new IllegalStateException("The state descriptor has not been initialized. " +
+					"Please call translate() before asking for state descriptors.");
+			}
+			List<ReducingStateDescriptor<Long>> descriptors = new ArrayList<>();
+			descriptors.add(this.afterFirstElementStateDesc);
+			return descriptors;
+		}
+
+		@Override
+		boolean onElement(Object element, long timestamp, P window, DslTriggerContext context) throws Exception {
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			Long timer = nextFiring.get();
+
+			if (timer == null) {
+				long nextTimer = context.getCurrentProcessingTime() + interval;
+				context.registerProcessingTimeTimer(nextTimer);
+				nextFiring.add(nextTimer);
+			} else {
+				// this is the case after a merge
+				context.registerProcessingTimeTimer(timer);
+			}
+			return false;
+		}
+
+		@Override
+		boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, P window, DslTriggerContext context) throws Exception {
+			if (isEventTimeTimer) {
+				return false;
+			}
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			Long timer = nextFiring.get();
+			return timer != null && timer == time;
+		}
+
+		@Override
+		void onFire(P window, DslTriggerContext context) throws Exception {
+			context.getPartitionedState(afterFirstElementStateDesc).clear();
+		}
+
+		@Override
+		public boolean canMerge() {
+			return true;
+		}
+
+		@Override
+		boolean onMerge(P window, DslTriggerContext context) throws Exception {
+			// the onElement will take care of setting a new timer
+
+			// We are in processing time. In the context of session windows,
+			// a new element can only expand the upper bound of a window.
+			// It cannot merge two already existing windows.
+			// So this trigger, the only thing it has to do is just
+			// reset the previous processing time timer, if there is any, or
+			// set a new timer because this is the first element.
+
+			context.mergePartitionedState(afterFirstElementStateDesc);
+			return false;
+		}
+
+		@Override
+		public String toString() {
+			return "ProcessingTimeTrigger.afterFirstElement(" + interval + " ms)";
+		}
+
+		@Override
+		void clear(P window, DslTriggerContext context) throws Exception {
+			ReducingState<Long> nextFiring = context.getPartitionedState(afterFirstElementStateDesc);
+			nextFiring.clear();
+		}
+	}
+
+	private class AfterEndOfWindow<P extends Window> extends DslTrigger<Object, P> {
+
+		private final DslTrigger<Object, P> earlyTrigger;
+
+		AfterEndOfWindow(DslTrigger<Object, P> earlyTrigger) {
+			this.earlyTrigger = earlyTrigger;
+		}
+
+		@Override
+		List<DslTrigger<Object, P>> getChildTriggers() {
+			return new ArrayList<>();
+		}
+
+		private DslTriggerInvokable<Object, P> getEarlyTriggerInvokable(DslTriggerContext ctx) {
+			if (earlyTrigger == null) {
+				return null;
+			}
+			return ctx.getChildInvokable(0);
+		}
+
+		@Override
+		DslTrigger<Object, P> translate(TypeSerializer<P> windowSerializer, long allowedLateness) {
+			return this;
+		}
+
+		@Override
+		<S extends State> List<StateDescriptor<S, ?>> getStateDescriptors() {
+			return Collections.emptyList();
+		}
+
+		@Override
+		boolean onElement(Object element, long timestamp, P window, DslTriggerContext context) throws Exception {
+			context.registerProcessingTimeTimer(window.maxTimestamp());
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			return earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnElement(element, timestamp, window);
+		}
+
+		@Override
+		boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			boolean shouldEarlyFire = earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeShouldFire(time, isEventTimeTimer, window);
+			return (!isEventTimeTimer && time == window.maxTimestamp()) || shouldEarlyFire;
+		}
+
+		@Override
+		void onFire(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeOnFire(window);
+			}
+		}
+
+		@Override
+		public boolean canMerge() {
+			return earlyTrigger == null || earlyTrigger.canMerge();
+		}
+
+		@Override
+		boolean onMerge(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			return earlyTriggerInvokable != null &&
+				earlyTriggerInvokable.invokeOnMerge(window);
+		}
+
+		@Override
+		public String toString() {
+			return "ProcessingTimeTrigger.afterEndOfWindow()" +
+				(earlyTrigger == null ? "" : ".withEarlyTrigger(" + earlyTrigger.toString() + ")");
+		}
+
+		@Override
+		void clear(P window, DslTriggerContext context) throws Exception {
+			DslTriggerInvokable<Object, P> earlyTriggerInvokable = getEarlyTriggerInvokable(context);
+			if (earlyTriggerInvokable != null) {
+				earlyTriggerInvokable.invokeClear(window);
+			}
+		}
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeat.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeat.java
new file mode 100644
index 00000000000..6e8d100dcc9
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeat.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.ByteSerializer;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+import java.util.Collections;
+import java.util.List;
+
+public class Repeat<T, W extends Window> extends DslTrigger<T, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final DslTrigger<T, W> trigger;
+
+	private ValueStateDescriptor<Byte> hasFiredStateDesc;
+
+	private Repeat(DslTrigger<T, W> trigger, boolean fireForever) {
+		this.trigger = trigger;
+		this.setIsRepeated(fireForever ? Repeated.FOREVER : Repeated.ONCE);
+	}
+
+	@Override
+	List<DslTrigger<T, W>> getChildTriggers() {
+		return Collections.singletonList(trigger);
+	}
+
+	@Override
+	DslTrigger<T, W> translate(TypeSerializer<W> windowSerializer, long allowedLateness) {
+		this.hasFiredStateDesc = this.isRepeated().once() ?
+			new ValueStateDescriptor<>("repeat-once", ByteSerializer.INSTANCE, (byte) 0) : null;
+		return this;
+	}
+
+	@Override
+	List<ValueStateDescriptor<Byte>> getStateDescriptors() {
+		return hasFiredStateDesc != null ?
+			Collections.singletonList(hasFiredStateDesc) :
+			Collections.<ValueStateDescriptor<Byte>>emptyList();
+	}
+
+	@Override
+	boolean shouldFireOnTimer(long time, boolean isEventTimeTimer, W window, DslTriggerContext context) throws Exception {
+		boolean shouldChildFire = context.getChildInvokable(0).invokeShouldFire(time, isEventTimeTimer, window);
+		if (hasFiredStateDesc == null) {
+			return shouldChildFire;
+		}
+
+		boolean hasFiredAlready = context.getPartitionedState(hasFiredStateDesc).value() == (byte) 1;
+		return !hasFiredAlready && shouldChildFire;
+	}
+
+	@Override
+	boolean onElement(T element, long timestamp, W window, DslTriggerContext context) throws Exception {
+		boolean childTriggerResult = context.getChildInvokable(0).invokeOnElement(element, timestamp, window);
+		if (hasFiredStateDesc == null) {
+			return childTriggerResult;
+		}
+
+		boolean hasFiredAlready = context.getPartitionedState(hasFiredStateDesc).value() == (byte) 1;
+		return !hasFiredAlready && childTriggerResult;
+	}
+
+	@Override
+	void onFire(W window, DslTriggerContext context) throws Exception {
+		context.getChildInvokable(0).invokeOnFire(window);
+		if (hasFiredStateDesc != null) {
+			context.getPartitionedState(hasFiredStateDesc).update((byte) 1);
+		}
+	}
+
+	@Override
+	void clear(W window, DslTriggerContext context) throws Exception {
+		context.getChildInvokable(0).invokeClear(window);
+		if (hasFiredStateDesc != null) {
+			context.getPartitionedState(hasFiredStateDesc).clear();
+		}
+	}
+
+	@Override
+	public boolean canMerge() {
+		return trigger.canMerge();
+	}
+
+	@Override
+	boolean onMerge(W window, DslTriggerContext context) throws Exception {
+		boolean childTriggerResult = context.getChildInvokable(0).invokeOnMerge(window);
+		if (hasFiredStateDesc != null) {
+			// even if we have fired for the old windows, we have not for the new
+			context.getPartitionedState(hasFiredStateDesc).clear();
+		}
+		return childTriggerResult;
+	}
+
+	@Override
+	public String toString() {
+		return hasFiredStateDesc != null ? trigger.toString() : "Repeat.Forever(" + trigger.toString() + ")";
+	}
+
+	public static <T, W extends Window> DslTrigger<T, W> Forever(DslTrigger<T, W> trigger) {
+		return new Repeat<>(trigger, true);
+	}
+
+	static <T, W extends Window> DslTrigger<T, W> Once(DslTrigger<T, W> trigger) {
+		return new Repeat<>(trigger, false);
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeated.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeated.java
new file mode 100644
index 00000000000..313fa8d8429
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/triggerdsl/Repeated.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.windowing.triggers.triggerdsl;
+
+/**
+ * Describes if the trigger is to fire once for a given key and window and then stop, or
+ * fire repeatedly, throughout the lifetime of the window. By default, triggers are <tt>Repeated.ONCE</tt>.
+ * To change this, use the {@link Repeat#Forever(DslTrigger)}, as in the following example:
+ * <pre>
+ *     {@code Repeat.Forever(EventTime.<TimeWindow>afterFirstElement(Time.milliseconds(100)))
+ * }</pre>
+ * The example above will fire every 100 milliseconds (in event time).
+ */
+public enum Repeated {
+
+	ONCE(false),
+
+	FOREVER(true),
+
+	UNDEFINED(null);
+
+	private final Boolean isRepeated;
+
+	Repeated(Boolean fireRepeatedly) {
+		this.isRepeated = fireRepeatedly;
+	}
+
+	public boolean isUndefined() {
+		return isRepeated == null;
+	}
+
+	public boolean once() {
+		return isRepeated != null && !isRepeated;
+	}
+
+	public boolean forever() {
+		return isRepeated != null && isRepeated;
+	}
+}
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 4de772920e4..249d773531a 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
@@ -1,4 +1,4 @@
-/**
+/*
 * 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
@@ -24,7 +24,6 @@
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.AppendingState;
-import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -118,7 +117,6 @@ public void merge(W mergeResult,
 								for (W m : mergedWindows) {
 									context.window = m;
 									context.clear();
-									deleteCleanupTimer(m);
 								}
 
 								// merge the merged state windows into the newly resulting state window
@@ -126,7 +124,7 @@ public void merge(W mergeResult,
 									stateWindowResult,
 									mergedStateWindows,
 									windowSerializer,
-									(StateDescriptor<? extends MergingState<?, ?>, ?>) windowStateDescriptor);
+									windowStateDescriptor);
 							}
 						});
 
@@ -162,10 +160,9 @@ public void merge(W mergeResult,
 				}
 
 				if (combinedTriggerResult.isPurge()) {
-					cleanup(actualWindow, windowState, mergingWindows);
-				} else {
-					registerCleanupTimer(actualWindow);
+					cleanupWindowContents(windowState);
 				}
+				registerCleanupTimer(actualWindow);
 			}
 
 		} else {
@@ -195,16 +192,17 @@ public void merge(W mergeResult,
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(window, windowState, null);
-				} else {
-					registerCleanupTimer(window);
+					cleanupWindowContents(windowState);
 				}
+				registerCleanupTimer(window);
 			}
 		}
 	}
 
 	@Override
 	public void processWatermark(Watermark mark) throws Exception {
+		this.currentWatermark = mark.getTimestamp();
+
 		boolean fire;
 		do {
 			Timer<K, W> timer = watermarkTimersQueue.peek();
@@ -237,7 +235,11 @@ public void processWatermark(Watermark mark) throws Exception {
 
 				Iterable<StreamRecord<IN>> contents = windowState.get();
 				if (contents == null) {
-					// if we have no state, there is nothing to do
+					// If we have no state, then if it is cleanup time, just clear the context as it may be that the trigger's
+					// state has not been cleared. This is useful especially in the discarding mode of the trigger.
+					if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+						cleanup(context.window, windowState, mergingWindows);
+					}
 					continue;
 				}
 
@@ -246,8 +248,19 @@ public void processWatermark(Watermark mark) throws Exception {
 					fire(context.window, contents);
 				}
 
-				if (triggerResult.isPurge() || (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+					// if it is cleanup time for the window, then cleanup everything
 					cleanup(context.window, windowState, mergingWindows);
+				} else if (triggerResult.isPurge()) {
+
+					// if we are on discarding mode and we are purging the state, then clear
+					// only the window contents and late the trigger state be.
+					// this is to avoid cleaning up the hasFiredOnTimeFlag in the EventTimeTrigger
+					// when we are operating on discarding mode. This could result in multiple
+					// firings if the watermark is equal to the end of the window and until it
+					// advances.
+
+					cleanupWindowContents(windowState);
 				}
 
 			} else {
@@ -256,8 +269,6 @@ public void processWatermark(Watermark mark) throws Exception {
 		} while (fire);
 
 		output.emitWatermark(mark);
-
-		this.currentWatermark = mark.getTimestamp();
 	}
 
 	@Override
@@ -299,7 +310,11 @@ public void trigger(long time) throws Exception {
 
 				Iterable<StreamRecord<IN>> contents = windowState.get();
 				if (contents == null) {
-					// if we have no state, there is nothing to do
+					// If we have no state, then if it is cleanup time, just clear the context as it may be that the trigger's
+					// state has not been cleared. This is useful especially in the discarding mode of the trigger.
+					if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+						cleanup(context.window, windowState, mergingWindows);
+					}
 					continue;
 				}
 
@@ -308,8 +323,10 @@ public void trigger(long time) throws Exception {
 					fire(context.window, contents);
 				}
 
-				if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
 					cleanup(context.window, windowState, mergingWindows);
+				} else if (triggerResult.isPurge()) {
+					cleanupWindowContents(windowState);
 				}
 
 			} else {
@@ -334,8 +351,24 @@ public IN apply(StreamRecord<IN> input) {
 				}
 			});
 		userFunction.apply(context.key, context.window, projectedContents, timestampedCollector);
+		context.onFire();
 	}
 
+	/**
+	 * Called when the window is to be cleaned up totally, i.e. when the allowed lateness expires. This
+	 * method clears up not only the window contents, but also all related metadata for the window, e.g.
+	 * the session length in case of session windows. After this method, the window is as if it never
+	 * existed in the system. The caller must ensure that the correct key is set in the state backend
+	 * and the context object.
+	 * <p/>
+	 * When operating on <tt>discarding()</tt> mode, then the {@link #cleanupWindowContents(AppendingState)}
+	 * is called, which only cleans up the window contents.
+	 *
+	 * @param window the window to be garbage collected.
+	 * @param windowState the contents of the window to be deleted.
+	 * @param mergingWindows the set of windows, in case of session windows,
+	 *                          to be deleted along with the main one.
+	 */
 	private void cleanup(W window,
 						ListState<StreamRecord<IN>> windowState,
 						MergingWindowSet<W> mergingWindows) throws Exception {
@@ -347,6 +380,16 @@ private void cleanup(W window,
 		context.clear();
 	}
 
+	/**
+	 * When operating on <tt>discarding()</tt> mode, this method cleans up <bb>only</bb>
+	 * the window contents, leaving the rest of the window metadata intact.
+	 *
+	 * @param windowState the state to be cleaned up.
+	 */
+	private void cleanupWindowContents(ListState<StreamRecord<IN>> windowState) throws Exception {
+		windowState.clear();
+	}
+
 	// ------------------------------------------------------------------------
 	// Getters for testing
 	// ------------------------------------------------------------------------
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 4e19c31c8d3..88ad48f71a8 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
@@ -191,7 +191,7 @@ public void merge(Collection<W> toBeMerged, W mergeResult) {
 
 			// don't merge the new window itself, it never had any state associated with it
 			// i.e. if we are only merging one pre-existing window into itself
-			// without extending the pre-exising window
+			// without extending the pre-existing window
 			if (!(mergedWindows.contains(mergeResult) && mergedWindows.size() == 1)) {
 				mergeFunction.merge(mergeResult,
 						mergedWindows,
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 e4939dbceb9..d3e79f3fc69 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
@@ -57,6 +57,7 @@
 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.Window;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
@@ -209,7 +210,7 @@ public WindowOperator(
 		this.keySerializer = checkNotNull(keySerializer);
 		this.windowStateDescriptor = windowStateDescriptor;
 		this.trigger = checkNotNull(trigger);
-		this.allowedLateness = allowedLateness;
+		this.allowedLateness = !windowAssigner.isEventTime() ? 0 : allowedLateness;
 
 		setChainingStrategy(ChainingStrategy.ALWAYS);
 	}
@@ -325,7 +326,6 @@ public void merge(W mergeResult,
 						for (W m: mergedWindows) {
 							context.window = m;
 							context.clear();
-							deleteCleanupTimer(m);
 						}
 
 						// merge the merged state windows into the newly resulting state window
@@ -369,10 +369,9 @@ public void merge(W mergeResult,
 				}
 
 				if (combinedTriggerResult.isPurge()) {
-					cleanup(actualWindow, windowState, mergingWindows);
-				} else {
-					registerCleanupTimer(actualWindow);
+					cleanupWindowContents(windowState);
 				}
+				registerCleanupTimer(actualWindow);
 			}
 		} else {
 			for (W window: elementWindows) {
@@ -400,16 +399,17 @@ public void merge(W mergeResult,
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(window, windowState, null);
-				} else {
-					registerCleanupTimer(window);
+					cleanupWindowContents(windowState);
 				}
+				registerCleanupTimer(window);
 			}
 		}
 	}
 
 	@Override
 	public void processWatermark(Watermark mark) throws Exception {
+		this.currentWatermark = mark.getTimestamp();
+
 		boolean fire;
 		do {
 			Timer<K, W> timer = watermarkTimersQueue.peek();
@@ -442,7 +442,11 @@ public void processWatermark(Watermark mark) throws Exception {
 
 				ACC contents = windowState.get();
 				if (contents == null) {
-					// if we have no state, there is nothing to do
+					// If we have no state, then if it is cleanup time, clear the context as it may be that the trigger's
+					// state has not been cleared. This is useful especially in the discarding mode of the trigger.
+					if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+						cleanup(context.window, windowState, mergingWindows);
+					}
 					continue;
 				}
 
@@ -451,8 +455,19 @@ public void processWatermark(Watermark mark) throws Exception {
 					fire(context.window, contents);
 				}
 
-				if (triggerResult.isPurge() || (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+					// if it is cleanup time for the window, then cleanup everything
 					cleanup(context.window, windowState, mergingWindows);
+				} else if (triggerResult.isPurge()) {
+
+					// if we are on discarding mode and we are purging the state, then clear
+					// only the window contents and late the trigger state be.
+					// this is to avoid cleaning up the hasFiredOnTimeFlag in the EventTimeTrigger
+					// when we are operating on discarding mode. This could result in multiple
+					// firings if the watermark is equal to the end of the window and until it
+					// advances.
+
+					cleanupWindowContents(windowState);
 				}
 
 			} else {
@@ -461,8 +476,6 @@ public void processWatermark(Watermark mark) throws Exception {
 		} while (fire);
 
 		output.emitWatermark(mark);
-
-		this.currentWatermark = mark.getTimestamp();
 	}
 
 	@Override
@@ -504,7 +517,11 @@ public void trigger(long time) throws Exception {
 
 				ACC contents = windowState.get();
 				if (contents == null) {
-					// if we have no state, there is nothing to do
+					// If we have no state, then if it is cleanup time, just clear the context as it may be that the trigger's
+					// state has not been cleared. This is useful especially in the discarding mode of the trigger.
+					if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
+						cleanup(context.window, windowState, mergingWindows);
+					}
 					continue;
 				}
 
@@ -513,8 +530,10 @@ public void trigger(long time) throws Exception {
 					fire(context.window, contents);
 				}
 
-				if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp)) {
 					cleanup(context.window, windowState, mergingWindows);
+				} else if (triggerResult.isPurge()) {
+					cleanupWindowContents(windowState);
 				}
 
 			} else {
@@ -524,9 +543,19 @@ public void trigger(long time) throws Exception {
 	}
 
 	/**
-	 * 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
-	 * correct key is set in the state backend and the context object.
+	 * Called when the window is to be cleaned up totally, i.e. when the allowed lateness expires. This
+	 * method clears up not only the window contents, but also all related metadata for the window, e.g.
+	 * the session length in case of session windows. After this method, the window is as if it never
+	 * existed in the system. The caller must ensure that the correct key is set in the state backend
+	 * and the context object.
+	 * <p/>
+	 * When operating on <tt>discarding()</tt> mode, then the {@link #cleanupWindowContents(AppendingState)}
+	 * is called, which only cleans up the window contents.
+	 *
+	 * @param window the window to be garbage collected.
+	 * @param windowState the contents of the window to be deleted.
+	 * @param mergingWindows the set of windows, in case of session windows,
+	 *                          to be deleted along with the main one.
 	 */
 	private void cleanup(W window,
 						AppendingState<IN, ACC> windowState,
@@ -538,6 +567,16 @@ private void cleanup(W window,
 		context.clear();
 	}
 
+	/**
+	 * When operating on <tt>discarding()</tt> mode, this method cleans up <bb>only</bb>
+	 * the window contents, leaving the rest of the window metadata intact.
+	 *
+	 * @param windowState the state to be cleaned up.
+	 */
+	private void cleanupWindowContents(AppendingState<IN, ACC> windowState) throws Exception {
+		windowState.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.
@@ -546,6 +585,7 @@ private void cleanup(W window,
 	private void fire(W window, ACC contents) throws Exception {
 		timestampedCollector.setAbsoluteTimestamp(window.maxTimestamp());
 		userFunction.apply(context.key, context.window, contents, timestampedCollector);
+		context.onFire();
 	}
 
 	/**
@@ -588,25 +628,15 @@ protected boolean isLate(W window) {
 	 * 					the window whose state to discard
 	 */
 	protected void registerCleanupTimer(W window) {
-		long cleanupTime = cleanupTime(window);
-		if (windowAssigner.isEventTime()) {
-			context.registerEventTimeTimer(cleanupTime);
-		} else {
-			context.registerProcessingTimeTimer(cleanupTime);
+		if (windowSerializer instanceof GlobalWindow.Serializer) {
+			return;
 		}
-	}
 
-	/**
-	 * Deletes the cleanup timer set for the contents of the provided window.
-	 * @param window
-	 * 					the window whose state to discard
-	 */
-	protected void deleteCleanupTimer(W window) {
 		long cleanupTime = cleanupTime(window);
 		if (windowAssigner.isEventTime()) {
-			context.deleteEventTimeTimer(cleanupTime);
+			context.registerEventTimeTimer(cleanupTime);
 		} else {
-			context.deleteProcessingTimeTimer(cleanupTime);
+			context.registerProcessingTimeTimer(cleanupTime);
 		}
 	}
 
@@ -701,7 +731,7 @@ public long getCurrentWatermark() {
 			try {
 				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateDescriptor);
 			} catch (Exception e) {
-				throw new RuntimeException("Could not retrieve state", e);
+				throw new RuntimeException("Could not retrieve state.", e);
 			}
 		}
 
@@ -772,6 +802,10 @@ public void deleteEventTimeTimer(long time) {
 			}
 		}
 
+		public void onFire() throws Exception {
+			trigger.onFire(window, this);
+		}
+
 		public TriggerResult onElement(StreamRecord<IN> element) throws Exception {
 			return trigger.onElement(element.getValue(), element.getTimestamp(), window, this);
 		}
@@ -966,4 +1000,9 @@ private void snapshotTimers(DataOutputView out) throws IOException {
 	public StateDescriptor<? extends AppendingState<IN, ACC>, ?> getStateDescriptor() {
 		return windowStateDescriptor;
 	}
+
+	@VisibleForTesting
+	public Tuple2<Integer, Integer> getNumberOfTimers() {
+		return new Tuple2<>(this.watermarkTimers.size(), this.processingTimeTimers.size());
+	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
index 7ac9e13355d..25b08a31539 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -331,7 +331,7 @@ public boolean canMerge() {
 		}
 	}
 
-	public static class DummyFolder implements FoldFunction<Tuple2<String, Integer>, Integer> {
+	private static class DummyFolder implements FoldFunction<Tuple2<String, Integer>, Integer> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTest.java
new file mode 100644
index 00000000000..9138b0a16c2
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerTest.java
@@ -0,0 +1,3464 @@
+/*
+ * 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.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+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.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction;
+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.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
+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.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.All;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.Any;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.Count;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTriggerRunner;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.EventTime;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.ProcessingTime;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.Repeat;
+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.InternalSingleValueWindowFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+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.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+
+public class TriggerTest {
+
+	private <W extends Window> WindowingTestHarness<String, Tuple2<String, Integer>, W> createTestHarness(
+		WindowAssigner<Object, W> assigner, DslTrigger<Object, W> trigger, long lateness) {
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		DslTriggerRunner<Object, W> runner = new DslTriggerRunner<>(trigger);
+		runner.createTriggerTree(assigner.getWindowSerializer(config), lateness);
+
+		return new WindowingTestHarness<>(config, assigner, keySelector, keyType, inputType, runner, lateness);
+	}
+
+	@Test
+	public void testEventTimeTriggerTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness =
+			createTestHarness(windowAssigner, EventTime.<TimeWindow>afterEndOfWindow().accumulating(), ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2000);								// +6 on_time
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);		// +7 because we get the default trigger
+
+		testHarness.processWatermark(2200);								// cleanup
+
+		// dropped
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		Assert.assertEquals(6 + 7 + 3, testHarness.getOutput().size());	// +3 for the watermarks
+
+		// check the we have processed all the timers
+		Assert.assertEquals(testHarness.getOperator().getNumberOfTimers(), new Tuple2<>(0, 0));
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerAfterFirstElementTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 200;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(EventTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.processWatermark(1100);								// +3
+
+		Assert.assertEquals(3 + 2, testHarness.getOutput().size());		// +2 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(1200);								// +4
+
+		Assert.assertEquals(7 + 3, testHarness.getOutput().size());		// +3 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.processWatermark(2000);								// +6
+
+		Assert.assertEquals(13 + 4, testHarness.getOutput().size());	// +4 for the watermark
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		Assert.assertEquals(13 + 4, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2100);								// +7
+
+		Assert.assertEquals(20 + 5, testHarness.getOutput().size());
+
+		// the following element is late but will be added,
+		// but the timer it will set will be after the cleanup (2200),
+		// so it will not fire.
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2201);
+
+		Assert.assertEquals(20 + 6, testHarness.getOutput().size()); // +6 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerAfterFirstElementGreaterThanWindowTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 200;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(EventTime.<TimeWindow>afterFirstElement(Time.milliseconds(4000))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// register timer for 5000
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2000);								// nothing to fire
+
+		Assert.assertEquals(2, testHarness.getOutput().size());			// +2 for the watermarks
+
+		// dropped as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.processWatermark(5000);								// no firing as we are after the cleanup
+
+		Assert.assertEquals(3, testHarness.getOutput().size());			// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			ProcessingTime.<TimeWindow>afterEndOfWindow().accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(2000);								// +6 on_time
+
+		Assert.assertEquals(6, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerAfterFirstElementTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(100);								//+3
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(210);								//+4
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(2000);							//+6
+
+		Assert.assertEquals(13, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerWithEarlyTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		DslTrigger<Object, TimeWindow> trigger = ProcessingTime.<TimeWindow>afterEndOfWindow()
+			.withEarlyTrigger(Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(10))))
+			.accumulating();
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			trigger,
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(10);								// +2 early firing
+
+		testHarness.setProcessingTime(12);
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(22);								// +5 early firing
+
+		Assert.assertEquals(2 + 5, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(2000);							// +12 = +6 for on_time
+																		//       +6 for early trigger from previous element
+
+		Assert.assertEquals(2 + 5 + 6 + 6, testHarness.getOutput().size() );
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerWithEarlyTumbling2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			ProcessingTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Repeat.Forever(Count.<TimeWindow>atLeast(2))),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +2 early firing
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +4 early firing
+
+		Assert.assertEquals(2 + 4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(20);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +6 early firing
+
+		Assert.assertEquals(2 + 4 + 6, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(2000);							// +6 for on_time
+
+		Assert.assertEquals(2 + 4 + 6 + 6, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerWithEarlyTumblingOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			ProcessingTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Count.<TimeWindow>atLeast(2)),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +2 early firing
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// no firing (ONCE)
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(20);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// no firing (ONCE)
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(2000);							// +6 for on_time
+
+		Assert.assertEquals(2 + 6, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerAfterFirstElementGreaterThanWindowTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingProcessingTimeWindows windowAssigner = TumblingProcessingTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(4000))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);				// will register timer for 4000
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(4000);									// here it should fire but window is cleaned up
+
+		Assert.assertEquals(0, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 300;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))))
+				.withLateTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(200)))
+				.accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+		testHarness.setProcessingTime(100);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(200);									// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());			// +1 for the watermark
+
+		testHarness.setProcessingTime(300);
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());			// no data, no firing
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(400);									// +5 early firing
+
+		Assert.assertEquals(3 + 5 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processWatermark(1999);									// +6 on-time firing
+
+		Assert.assertEquals(3 + 5 + 6 + 2, testHarness.getOutput().size());	// +2 for watermarks
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// after the recovery, the watermark has been set back to Long.MIN_VALUE
+		// so this also checks that the hasFiredOnTime flag does its job
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		testHarness.setProcessingTime(500);									// no firing as we are in the late period
+		Assert.assertEquals(3 + 5 + 6 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(600);									// +10 late firing
+		Assert.assertEquals(3 + 5 + 6 + 10 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.setProcessingTime(800);									// no late firing because it is repeat.Once
+
+		Assert.assertEquals(3 + 5 + 6 + 10 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(2299);									// this is cleanup
+
+		// dropped due to lateness
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		Assert.assertEquals(3 + 5 + 6 + 10 + 3, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateTumblingDiscarding() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 300;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))))
+				.withLateTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(200)))
+				.discarding(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.processWatermark(1000);
+		testHarness.setProcessingTime(100);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(10L, 10L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(200);									// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());			// +1 for the watermark
+
+		testHarness.setProcessingTime(300);
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());			// no data, no firings
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+		testHarness.setProcessingTime(400);									// +2 early firing FOREVER
+
+		Assert.assertEquals(3 + 2 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processWatermark(2100);									// +1 on-time firing
+
+		Assert.assertEquals(3 + 2 + 1 + 2, testHarness.getOutput().size());	// +1 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		testHarness.setProcessingTime(500);									// no firing as we are in the late period
+		Assert.assertEquals(3 + 2 + 1 + 2, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(10L, 11L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(600);									// +4 late firing
+		Assert.assertEquals(3 + 2 + 1 + 4 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.setProcessingTime(800);									// no late firing because we are at ONCE
+
+		testHarness.processWatermark(2299); // this is cleanup
+
+		// dropped due to lateness
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		Assert.assertEquals(3 + 2 + 1 + 4 + 3, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyOnlyTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 300;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Repeat.Forever(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))))
+				.accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.processWatermark(1000);
+		testHarness.setProcessingTime(100);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		testHarness.setProcessingTime(200);										// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());				// +1 for the watermark
+
+		testHarness.setProcessingTime(300);
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());				// no data, no firing
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1500);
+		testHarness.setProcessingTime(400);										// +5 early firing
+
+		Assert.assertEquals(3 + 5 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processWatermark(2100);										// + 6 on-time firing
+
+		Assert.assertEquals(3 + 5 + 6 + 2, testHarness.getOutput().size());		// +1 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);				// +7 firing late element
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2100);										// TODO: 9/24/16 this is to simulate that we knwo the last watermark after the failure
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);				// +8 firing at every late element
+
+		testHarness.setProcessingTime(500);										// no firing because we have no late trigger
+		Assert.assertEquals(3 + 5 + 6 + 7 + 8 + 3, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(600);										// no firing because we have no late trigger
+		Assert.assertEquals(3 + 5 + 6 + 7 + 8 + 3, testHarness.getOutput().size());
+
+		testHarness.processWatermark(2301);										// this is cleanup
+
+		// dropped due to lateness
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		Assert.assertEquals(3 + 5 + 6 + 7 + 8 + 4, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithLateOnlyTumbling() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 300;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withLateTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(200)))
+				.accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+		testHarness.processWatermark(1000);
+		testHarness.setProcessingTime(100);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(200);                            // no early trigger, no firing
+
+		Assert.assertEquals(1, testHarness.getOutput().size());        // +1 for the watermark
+
+		testHarness.setProcessingTime(300);
+
+		Assert.assertEquals(1, testHarness.getOutput().size());        // no firing
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2100);                            // +4 on-time firing
+
+		Assert.assertEquals(4 + 2, testHarness.getOutput().size());    // +2 for the watermarks
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		// do a snapshot, close and restore
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(500);                            // + 8 late firing
+		Assert.assertEquals(4 + 8 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(600);                            // no firing (ONCE)
+		Assert.assertEquals(4 + 8 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(2299);								// this is cleanup
+
+		// dropped due to lateness
+		testHarness.processElement(new Tuple2<>("key1", 1), 1900);
+
+		Assert.assertEquals(4 + 8 + 3, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAnyOfTriggerWithTumblingWindows() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		// this also tests that the accumulating/discarding mode of the parent overrides that of the child
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Any.of(Count.<TimeWindow>atLeast(4), Count.<TimeWindow>atLeast(3)).discarding()).accumulating(),
+			0);
+
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +3 count_3 fires
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);	// no firing because both counters are reset
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +6 count_3 fires
+
+		Assert.assertEquals(9, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	//
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// no firing
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +9 count_3 fires
+
+		testHarness.processWatermark(1999);
+
+		Assert.assertEquals(18 + 1, testHarness.getOutput().size());// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAnyOfTriggerWithTumblingWindows2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Any.of(Count.<TimeWindow>atLeast(4), ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100)))).accumulating(),
+			0);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(10);
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+
+		testHarness.setProcessingTime(110);							// +2 processing time
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);	// no firing because both states are cleaned at previous firing
+																	// implicitly all triggers are "after previous firing"
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +6 count_4 fires (accumulating)
+
+		Assert.assertEquals(2 + 6, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(250);							// no firing, state has been cleared
+
+		Assert.assertEquals(2 + 6, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(350);							// +9, processing time
+
+		Assert.assertEquals(2 + 6 + 9, testHarness.getOutput().size());
+
+		testHarness.processWatermark(2000);							// the cleanup timer
+
+		// check the we have processed all the timers
+		// (we do not have separate cleanup timer because lateness=0)
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAnyOfTriggerWithTumblingWindows3() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Any.of(Count.<TimeWindow>atLeast(4), EventTime.<TimeWindow>afterEndOfWindow())).accumulating(),
+			0);
+
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);		// +4 count_4 fires
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(1999);								// +6 on-time firing
+
+		Assert.assertEquals(10 + 1, testHarness.getOutput().size());	// +1 for the watermark
+
+		// check the we have processed all the timers
+		// (we do not have separate cleanup timer because lateness=0)
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAllOfTriggerWithTumblingWindows() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(All.of(Count.<TimeWindow>atLeast(4), Count.<TimeWindow>atLeast(2))).accumulating(),
+			0);
+
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);		// +0 count_2 fires but ignored
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// +4 both fire here
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);		// +0 count_2 fires but ignored
+
+		Assert.assertEquals(4, testHarness.getOutput().size());			// +1 for the watermark
+
+		testHarness.processWatermark(1999);								// this will lead to processing the cleanup timer
+
+		Assert.assertEquals(4 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAllOfTriggerWithTumblingWindowsDiscarding() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(All.of(Count.<TimeWindow>atLeast(3), Count.<TimeWindow>atLeast(2))).discarding(),
+			0);
+
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);		// +0 count_2 fires but ignored
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);		// +3 all fire here
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);		// +3 all fire here
+
+		Assert.assertEquals(6, testHarness.getOutput().size());			// +1 for the watermark
+
+		testHarness.processWatermark(1999);								// this will lead to processing the cleanup timer
+
+		Assert.assertEquals(6 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAllOfTriggerWithTumblingWindows2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(All.of(Count.<TimeWindow>atLeast(4), EventTime.<TimeWindow>afterEndOfWindow())).accumulating(),
+			0);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// the 4 fires here but is ignored
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1006);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1007);
+
+		testHarness.processWatermark(1999);								// +8 on-time firing with >4 elements
+
+		Assert.assertEquals(8 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAllOfTriggerWithTumblingWindowsOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			All.of(Count.<TimeWindow>atLeast(4), Count.<TimeWindow>atLeast(5)).accumulating(),
+			0);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// the 4 fires here but is ignored
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);		// both fire here
+
+		Assert.assertEquals(5, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1006);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1007);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1007);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1007);		// both would fire here but we are at ONCE
+
+		Assert.assertEquals(5, testHarness.getOutput().size());
+
+		testHarness.processWatermark(1999);								// cleanup
+
+		Assert.assertEquals(5 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateTumbling2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final long allowedLateness = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Count.<TimeWindow>atLeast(4))
+				.withLateTrigger(Count.<TimeWindow>atLeast(3))
+				.accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 800);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1002);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1003);	// +4 early for key1
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +4 early firing for key2
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		Assert.assertEquals(4 + 4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1006);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1007);
+
+		testHarness.processWatermark(1999);							// +10 on-time firing
+
+		Assert.assertEquals(4 + 4 + 10 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// late but accepted elements
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1901);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1902);	// +9 late firing for key2
+
+		Assert.assertEquals(4 + 4 + 10 + 9 + 1, testHarness.getOutput().size());	// +1 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1903);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1904);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1905);	// +7 late firing for key1
+
+		Assert.assertEquals(4 + 4 + 10 + 9 + 7 + 1, testHarness.getOutput().size());    // +1 for the watermark
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1904);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1905);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1904);	// no more late firings
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1906);
+
+		testHarness.processWatermark(2200);
+
+		Assert.assertEquals(4 + 4 + 10 + 9 + 7 + 2, testHarness.getOutput().size()); // +2 for the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyOnlyTumbling2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final long allowedLateness = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		Count<TimeWindow> countTrigger = Count.atLeast(4);
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(countTrigger)
+				.withLateTrigger(Repeat.Forever(countTrigger))
+				.accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 800);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// + 4 early firing
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// no firing
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		testHarness.processWatermark(1999);								// +10 on-time
+
+		Assert.assertEquals(4 + 10 + 1, testHarness.getOutput().size());
+
+		// late but accepted elements
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);		// + 14 late firing
+
+		Assert.assertEquals(4 + 14 + 10 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);		// + 18 late firing
+
+		Assert.assertEquals(4 + 14 + 18 + 10 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 2150);
+		testHarness.processWatermark(10999);							// fire just the new window
+
+		Assert.assertEquals(4 + 14 + 18 + 10 + 1 + 2, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyOnlyTumbling3() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final long allowedLateness = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(
+				EventTime.<TimeWindow>afterEndOfWindow()
+					.withEarlyTrigger(Count.<TimeWindow>atLeast(4)))
+				.accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 800);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// + 4 early firing
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);		// + 8 early firing
+
+		Assert.assertEquals(4 + 8, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(1999);								// +10 on-time
+
+		Assert.assertEquals(4 + 8 + 10 + 1, testHarness.getOutput().size());
+
+		// late but accepted elements
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);		// +11 late firing
+
+		Assert.assertEquals(4 + 8 + 10 + 11 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 2150);
+		testHarness.processWatermark(10999);							// fire just the new window
+
+		Assert.assertEquals(4 + 8 + 10 + 11 + 1 + 2, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithLateOnlyTumbling2() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final long allowedLateness = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withLateTrigger(Count.<TimeWindow>atLeast(4)),
+			allowedLateness);
+
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 800);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		testHarness.processWatermark(1999);							// +6 on-time
+
+		Assert.assertEquals(6 + 1, testHarness.getOutput().size());	// +1 for the watermark
+
+		// late but accepted elements
+		testHarness.processElement(new Tuple2<>("key2", 1), 1900);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1901);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1902);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1903);	// +10 late firing
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1902);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1903);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1902);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1903);	// no more late firings
+
+		testHarness.processWatermark(2200);
+
+		Assert.assertEquals(16 + 2, testHarness.getOutput().size()); // +2 for the watermarks
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	// SESSION WINDOW TESTING
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateSessionMerging() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 5000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(
+				EventTime.<TimeWindow>afterEndOfWindow()
+					.withEarlyTrigger(Count.<TimeWindow>atLeast(3))
+					.withLateTrigger(Count.<TimeWindow>atLeast(5))
+			).accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);		// + 3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(7500);								// +3 onTime firing, with no new data
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);		// late counter for old session is 1
+
+																		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// early of new session is 2
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+																		// MERGE the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);		// here we must have a firing of the
+																		// early trigger of the merged window
+																		// because the new early count is 4		+7
+
+		Assert.assertEquals(3 + 3 + 7 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);		// early firing							+10
+
+		Assert.assertEquals(3 + 3 + 7 + 10 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(15000);							// +10 onTime of new session, with no new data
+
+		testHarness.processWatermark(20000);							// also process the last cleanup timer
+
+		Assert.assertEquals(3 + 3 + 7 + 10 + 10 + 4, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateSessionMergingOnce() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 5000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Count.<TimeWindow>atLeast(3))
+				.withLateTrigger(Count.<TimeWindow>atLeast(5)),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);		// + 3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());		// +1 for the watermark
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(7500);								// +3 onTime firing, with no new data
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);		// + 8 late firing
+
+		Assert.assertEquals(3 + 3 + 8 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);		// late of old session is 1
+
+		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// early of new session is 2
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// MERGE the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);		// here we must have a firing of the
+		// early trigger of the merged window
+		// because the new early count is 4		+7
+
+		Assert.assertEquals(3 + 3 + 8 + 12 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9000);		// early firing							+15
+
+		Assert.assertEquals(3 + 3 + 8 + 12 + 15 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(15000);							// +15 onTime of new session, with no new data
+
+		testHarness.processWatermark(20000);							// also process the last cleanup timer
+
+		Assert.assertEquals(3 + 3 + 8 + 12 + 15 + 15 + 4, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateSessionMerging2() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(
+				EventTime.<TimeWindow>afterEndOfWindow()
+					.withEarlyTrigger(Count.<TimeWindow>atLeast(3))
+					.withLateTrigger(Count.<TimeWindow>atLeast(5))
+			).accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);		// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(7500);								// +3 early firing with no new element
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);		// late of old session is 1
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// early of new session is 2
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(11500);							// + 2 onTime firing for new session
+
+		Assert.assertEquals(3 + 3 + 2 + 3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// late of new session is 1
+		testHarness.processElement(new Tuple2<>("key2", 1), 9500);		// early of expanded new session is 2
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9500);		// + 5 early of new session is 3
+		Assert.assertEquals(3 + 3 + 2 + 3 + 5, testHarness.getOutput().size());
+
+		// this will merge the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);		// the new early count is 2 (one was the late in the old session)
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// +11  early firing of new merged session
+
+		Assert.assertEquals(3 + 3 + 2 + 3 + 5 + 11, testHarness.getOutput().size());
+
+		testHarness.processWatermark(12500);							// +11 onTime of new session, with no new data
+
+		Assert.assertEquals(3 + 3 + 2 + 3 + 5 + 11 + 11 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// +16 late firing
+
+		Assert.assertEquals(3 + 3 + 2 + 3 + 5 + 11 + 11 + 16 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(25000);
+
+		//nothing happens at cleanup, we just add the last watermark
+		Assert.assertEquals(3 + 3 + 2 + 3 + 5 + 11 + 11 + 16 + 1 + 1, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyOnlySessionMerging() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow()
+				.withEarlyTrigger(Count.<TimeWindow>atLeast(3)),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(4998);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);		// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(7500);								// +3 onTime firing with no new element
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);		// +4 late firing	old_counter = 0
+
+		Assert.assertEquals(3 + 3 + 4 + 2, testHarness.getOutput().size());
+
+		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// early of new session is 2
+		testHarness.processWatermark(11500);							// +2 onTime firing
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// +3 late firing		new_counter = 0 (we fired)
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9500);		//	no firing because the session got extended	new_counter = 1
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(11500);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9500);		//							new_counter = 2
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 9500);		// +6 early firing			new_counter = 3
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6, testHarness.getOutput().size());
+
+		// this will merge the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);		//							new_counter = 1
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		//							new_counter = 2
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		// +13	early firing		new_counter = 3 (because it is a new session so the ONCE does not apply)
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		//							new_counter = 1
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6 + 13, testHarness.getOutput().size());
+
+		testHarness.processWatermark(12500);							// +14 on-time firing
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6 + 13 + 14 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);		//	+15 late firing
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6 + 13 + 14 + 15 + 1, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(12500);						// TODO: 9/24/16 this is to simulate that we knwo the last watermark after the failure
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);	//	no late firing					count = 2
+
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6 + 13 + 14 + 15 + 16 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(25000);
+
+		//nothing happens at cleanup, we just add the last watermark
+		Assert.assertEquals(3 + 3 + 4 + 2 + 3 + 4 + 6 + 13 + 14 + 15 + 16 + 3, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateProcessingSessionMerging() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(
+				EventTime.<TimeWindow>afterEndOfWindow()
+					.withEarlyTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.seconds(1)))
+					.withLateTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.seconds(2)))),
+			allowedLateness);
+
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(0);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+
+		testHarness.setProcessingTime(1000);								// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(2000);								// no firing, no data
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(7500);									// no firing, no data
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// late of old session registers timer for 3000
+
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(2500);
+
+		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);			// early of new session registers timer for 3500
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// this will merge the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);			// the new early timer has to be 3000
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(3000);								// +7 early firing of the merged session
+
+		Assert.assertEquals(3 + 3 + 2 + 7, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(3700);								// the 3500 timer should be ignored because its window was merged
+
+		Assert.assertEquals(3 + 3 + 2 + 7, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(12500);								// +7 firing with no new element
+
+		Assert.assertEquals(3 + 3 + 2 + 7 + 7 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);			// late registers timer for 5700
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(5800);
+
+		Assert.assertEquals(3 + 3 + 2 + 7 + 7 + 12 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(25000);
+
+		//nothing happens at cleanup, we just add the last watermark
+		Assert.assertEquals(3 + 3 + 2 + 7 + 7 + 12 + 2, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyAndLateMixSessionMerging() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(
+				EventTime.<TimeWindow>afterEndOfWindow()
+					.withEarlyTrigger(ProcessingTime.<TimeWindow>afterFirstElement(Time.seconds(1)))
+					.withLateTrigger(Count.<TimeWindow>atLeast(3))
+			).accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(0);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+
+		testHarness.setProcessingTime(1000);								// +3 early firing
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(2000);								// no firing, no data
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);			// registers timer for 3000
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(7500);									// +4 onTime firing
+
+		Assert.assertEquals(3 + 4 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(3000);								// this should not fire because we had an on-time firing
+		Assert.assertEquals(3 + 4 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// late of old session is 1
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// late of old session is 2
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// +7 late firing of old session
+
+		Assert.assertEquals(3 + 4 + 7 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// late of old session is 1 and register timer for 4000
+
+		testHarness.setProcessingTime(5000);								// nothing here as we are in the late phase
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 3000);			// late of old session is 2 and register timer for 6000
+
+		Assert.assertEquals(3 + 4 + 7 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(5500);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// new session
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);			// early of new session registers timer for 6500
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		// this will merge the two sessions into one
+		testHarness.processElement(new Tuple2<>("key2", 1), 7000);			// this should keep the 6000 timer after merging
+
+		testHarness.setProcessingTime(6000);								// +12 early firing of merged session
+
+		Assert.assertEquals(3 + 4 + 7 + 12 + 2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(6700);								// nothing should happen here
+
+		Assert.assertEquals(3 + 4 + 7 + 12 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(12500);								// +12 on time firing with no new element
+
+		Assert.assertEquals( 3 + 4 + 7 + 12 + 12 + 3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);			// +15 late firing
+
+		testHarness.setProcessingTime(10000);
+
+		Assert.assertEquals(3 + 4 + 7 + 12 + 12 + 15 + 3, testHarness.getOutput().size());
+
+		testHarness.processWatermark(25000);
+
+		//nothing happens at cleanup, we just add the last watermark
+		Assert.assertEquals(3 + 4 + 7 + 12 + 12 + 15 + 4, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testOnceTriggerInSessions() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Count.<TimeWindow>atLeast(3),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		Assert.assertEquals(new Tuple2<>(1, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 13999
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+		Assert.assertEquals(new Tuple2<>(2, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 14999
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);			// +3 early firing
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(3, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 17499
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);			// no firing - we are in ONCE mode
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+
+		Assert.assertEquals(new Tuple2<>(3, 0), testHarness.getOperator().getNumberOfTimers());	// the cleanup is already there so no re-adding
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(7500);									// nothing because we only have a count trigger
+		Assert.assertEquals(3 + 2, testHarness.getOutput().size());
+
+		// new session
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 7501);										// new session with one element
+		Assert.assertEquals(new Tuple2<>(4, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 20500
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8000);
+		Assert.assertEquals(new Tuple2<>(5, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 20999
+
+		Assert.assertEquals(3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8000);			// +3 for new session
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		Assert.assertEquals(new Tuple2<>(6, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 21499
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);			// +6 because these elements expand the window
+
+		Assert.assertEquals(3 + 3 + 6 + 2, testHarness.getOutput().size());	// no firing
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		Assert.assertEquals(3 + 3 + 6 + 2, testHarness.getOutput().size());	// no firing because we are in ONCE mode
+
+		// merging the two sessions
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 6500);
+		Assert.assertEquals(new Tuple2<>(7, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 21499 but for the whole window 1000 to 11500, so reinserted
+
+		Assert.assertEquals(3 + 3 + 6 + 17 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(21500);								// to process all the cleanup timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testOnceTriggerInSessionsDiscarding() throws Exception {
+		final int GAP_SIZE = 3;
+		final long allowedLateness = 10000; // greater than the gap
+
+		EventTimeSessionWindows windowAssigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Count.<TimeWindow>atLeast(3).discarding(),
+			allowedLateness);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		Assert.assertEquals(new Tuple2<>(1, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 13999
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 2000);
+		testHarness.processWatermark(4998);
+		Assert.assertEquals(new Tuple2<>(2, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 14999
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);										// +3 firing
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		Assert.assertEquals(new Tuple2<>(3, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 17499
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 4500);
+
+		Assert.assertEquals(new Tuple2<>(3, 0), testHarness.getOperator().getNumberOfTimers());	// the timer is already there
+
+		Assert.assertEquals(3 + 1, testHarness.getOutput().size());
+
+		testHarness.processWatermark(7500);																// nothing because we only have a count trigger
+		Assert.assertEquals(3 + 2, testHarness.getOutput().size());
+
+		// new session
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 7501);										// new session with one element
+		Assert.assertEquals(new Tuple2<>(4, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 20500
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8000);
+		Assert.assertEquals(new Tuple2<>(5, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 20999
+
+		Assert.assertEquals(3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8000);										// +3 for new session
+		Assert.assertEquals(3 + 3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		Assert.assertEquals(new Tuple2<>(6, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 21499
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);										// +3 because these elements expand the window
+
+		Assert.assertEquals(3 + 3 + 3 + 2, testHarness.getOutput().size());								// no firing
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+		testHarness.processElement(new Tuple2<>("key2", 1), 8500);
+
+		Assert.assertEquals(3 + 3 + 3 + 2, testHarness.getOutput().size());								// no firing because we are in ONCE mode
+
+		// merging the two sessions
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 6500);
+		Assert.assertEquals(new Tuple2<>(7, 0), testHarness.getOperator().getNumberOfTimers());	// cleanup for 21499 but for the whole window 1000 to 11500, so reinserted
+
+		Assert.assertEquals(3 + 3 + 3 + 8 + 2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(21500);															// to process all the cleanup timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	//			End of Session window testing
+
+	@Test
+	public void testStateDisambiguation() throws Exception {
+		final int WINDOW_SIZE = 2;
+		final long allowedLateness = 0; // greater than the gap
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE));
+
+		Count<TimeWindow> count = Count.atLeast(5);
+		Any<TimeWindow> child = Any.of(count, count, count);
+		Any<TimeWindow> child2 = Any.of(child, child, count);
+		Any<TimeWindow> root = Any.of(child2, child2);
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(root).accumulating(),
+			allowedLateness);
+
+		testHarness.open();
+
+		// if we had collision, then we should have each element increasing the counter twice
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 100);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 100);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 100);	// so without disambiguation here we would fire
+		testHarness.processWatermark(2001);
+
+		Assert.assertEquals(1, testHarness.getOutput().size());		// just the watermark
+
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	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;
+		}
+	}
+
+	// Different state types tests.
+
+	@Test
+	public void testFoldingState() throws Exception {
+		final int WINDOW_SIZE = 2;
+		final long LATENESS = 100;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		FoldingStateDescriptor<Tuple2<String, Integer>, Tuple2<String, Integer>> windowStateDesc =
+			new FoldingStateDescriptor<>(
+				"window-contents",
+				new Tuple2<>((String) null, 0),
+				new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+					@Override
+					public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator, Tuple2<String, Integer> value) throws Exception {
+						return new Tuple2<>(value.f0, accumulator.f1 + value.f1);
+					}
+				},
+				inputType.createSerializer(config));
+
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS));
+		DslTriggerRunner runner = new DslTriggerRunner<>(Repeat.Forever(EventTime.<TimeWindow>afterEndOfWindow()).discarding());
+		runner.createTriggerTree(assigner.getWindowSerializer(config), LATENESS);
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
+			new WindowOperator<>(
+				assigner,
+				new TimeWindow.Serializer(),
+				keySelector,
+				keyType.createSerializer(config),
+				windowStateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughFunction()),
+				runner,
+				LATENESS);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, keySelector, keyType);
+
+		operator.setInputType(inputType, config);
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		// normal element
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 1000));
+		testHarness.processWatermark(new Watermark(1599));
+
+		expected.add(new Watermark(1599));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 2), 1999));
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 3), 1999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processWatermark(new Watermark(1999));
+
+		expected.add(new Watermark(1999)); // here it fires and purges
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 1999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 1000)); // + 1 we have a late firing because we get the default trigger
+		testHarness.processWatermark(new Watermark(2100));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 2000));
+		testHarness.processWatermark(new Watermark(5000));
+
+		expected.add(new Watermark(2100)); // here is the cleanup timer
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 3999));
+		expected.add(new Watermark(5000));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	@Test
+	public void testCleanupTimerWithEmptyReduceStateForTumblingWindows() throws Exception {
+		final int WINDOW_SIZE = 2;
+		final long LATENESS = 100;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+			new WindowOperatorTest.SumReducer(),
+			inputType.createSerializer(config));
+
+		TumblingEventTimeWindows assigner = TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS));
+		DslTriggerRunner runner = new DslTriggerRunner<>(Repeat.Forever(EventTime.afterEndOfWindow()).discarding());
+		runner.createTriggerTree(assigner.getWindowSerializer(config), LATENESS);
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
+			new WindowOperator<>(
+				assigner,
+				new TimeWindow.Serializer(),
+				keySelector,
+				keyType.createSerializer(config),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				runner,
+				LATENESS);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, keySelector, keyType);
+
+		operator.setInputType(inputType, config);
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		// normal element
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processWatermark(new Watermark(1599));
+		testHarness.processWatermark(new Watermark(1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 1000));  // + 1 we have a late firing because we get the default trigger
+		testHarness.processWatermark(new Watermark(2000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 2000));
+		testHarness.processWatermark(new Watermark(5000));
+
+		expected.add(new Watermark(1599));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 2), 1999));
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 1999));
+		expected.add(new Watermark(1999)); // here it fires and purges
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 1999));
+		expected.add(new Watermark(2000)); // here is the cleanup timer
+		expected.add(new StreamRecord<>(new Tuple2<>("key1", 1), 3999));
+		expected.add(new Watermark(5000));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	private class PassThroughFunction implements WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void apply(String k, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<Tuple2<String, Integer>> out) throws Exception {
+			for (Tuple2<String, Integer> in : input) {
+				out.collect(in);
+			}
+		}
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSlidingEventTimeWindowsReduce() throws Exception {
+		final int WINDOW_SIZE = 3;
+		final int WINDOW_SLIDE = 1;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+			new WindowOperatorTest.SumReducer(),
+			inputType.createSerializer(config));
+
+		SlidingEventTimeWindows assigner = SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS));
+		DslTriggerRunner runner = new DslTriggerRunner<>(Repeat.Forever(EventTime.afterEndOfWindow()).discarding());
+		runner.createTriggerTree(assigner.getWindowSerializer(config), 0);
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+			assigner,
+			new TimeWindow.Serializer(),
+			keySelector,
+			keyType.createSerializer(config),
+			stateDesc,
+			new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+			runner,
+			0);
+
+		operator.setInputType(inputType, config);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, keySelector, keyType);
+
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+
+		testHarness.processWatermark(new Watermark(999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 999));
+		expectedOutput.add(new Watermark(999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(1999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 1999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 1999));
+		expectedOutput.add(new Watermark(1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new Watermark(2999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.setup();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(new Watermark(3999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 5), 3999));
+		expectedOutput.add(new Watermark(3999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(4999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 4999));
+		expectedOutput.add(new Watermark(4999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
+		expectedOutput.add(new Watermark(5999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		// those don't have any effect...
+		testHarness.processWatermark(new Watermark(6999));
+		testHarness.processWatermark(new Watermark(7999));
+		expectedOutput.add(new Watermark(6999));
+		expectedOutput.add(new Watermark(7999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testCleanupTimerWithEmptyReduceStateForSessionWindows() throws Exception {
+
+		final int GAP_SIZE = 3;
+		final long LATENESS = 10;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+			new WindowOperatorTest.SumReducer(), inputType.createSerializer(config));
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+		DslTriggerRunner runner = new DslTriggerRunner<>(Repeat.Forever(EventTime.afterEndOfWindow()).accumulating());
+		runner.createTriggerTree(assigner.getWindowSerializer(config), LATENESS);
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
+			new WindowOperator<>(
+				assigner,
+				new TimeWindow.Serializer(),
+				keySelector,
+				keyType.createSerializer(config),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new WindowOperatorTest.ReducedSessionWindowFunction()),
+				runner,
+				LATENESS);
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), config);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, keySelector, keyType);
+
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processWatermark(new Watermark(3999));
+
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-2", 1000L, 4000L), 3999));
+		expected.add(new Watermark(3999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 2000));
+		testHarness.processWatermark(new Watermark(4999));
+
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-3", 1000L, 5000L), 4999));
+		expected.add(new Watermark(4999));
+
+		testHarness.processWatermark(new Watermark(14600));
+		expected.add(new Watermark(14600));
+
+		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new WindowOperatorTest.Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	@Test
+	public void testCleanupTimerWithEmptyReduceStateForSessionWindowsDiscarding() throws Exception {
+
+		final int GAP_SIZE = 3;
+		final long LATENESS = 10;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+			new WindowOperatorTest.SumReducer(), inputType.createSerializer(config));
+
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		EventTimeSessionWindows assigner = EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE));
+		DslTriggerRunner runner = new DslTriggerRunner<>(Repeat.Forever(EventTime.<TimeWindow>afterEndOfWindow()).discarding());
+		runner.createTriggerTree(assigner.getWindowSerializer(config), LATENESS);
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
+			new WindowOperator<>(
+				assigner,
+				new TimeWindow.Serializer(),
+				keySelector,
+				keyType.createSerializer(config),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new WindowOperatorTest.ReducedSessionWindowFunction()),
+				runner,
+				LATENESS);
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), config);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, keySelector, keyType);
+
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+		testHarness.processWatermark(new Watermark(3999));
+
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-2", 1000L, 4000L), 3999));
+		expected.add(new Watermark(3999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 2000));
+		testHarness.processWatermark(new Watermark(4999));
+
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 1000L, 5000L), 4999));	// because the previous window has not been gc'ed yet,
+																						// we are still in the allowed lateness
+		expected.add(new Watermark(4999));
+
+		testHarness.processWatermark(new Watermark(14600));
+		expected.add(new Watermark(14600));
+
+		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new WindowOperatorTest.Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	// Global windows testing.
+
+	@Test
+	public void testEventTimeTriggerGlobal() throws Exception {
+		final int ALLOWED_LATENESS = 100;
+
+		GlobalWindows windowAssigner = GlobalWindows.create();
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, GlobalWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(EventTime.<GlobalWindow>afterFirstElement(Time.milliseconds(100))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(1100);                            // + 6
+
+		Assert.assertEquals(6 + 2, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		testHarness.processWatermark(1200);                            // +7
+
+		Assert.assertEquals(6 + 7 + 3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		testHarness.processWatermark(1300);                            // + 9
+
+		Assert.assertEquals(6 + 7 + 9 + 4, testHarness.getOutput().size()); // +3 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testProcessingTimeTriggerGlobal() throws Exception {
+		final int ALLOWED_LATENESS = 100;
+
+		GlobalWindows windowAssigner = GlobalWindows.create();
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, GlobalWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(ProcessingTime.<GlobalWindow>afterFirstElement(Time.milliseconds(100))).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(1100);                            // + 6
+
+		Assert.assertEquals(6, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		testHarness.setProcessingTime(1200);                            // +7
+
+		Assert.assertEquals(6 + 7, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		testHarness.setProcessingTime(1300);                            // + 9
+
+		Assert.assertEquals(6 + 7 + 9, testHarness.getOutput().size());
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testCountTriggerGlobal() throws Exception {
+		final int ALLOWED_LATENESS = 100;
+
+		GlobalWindows windowAssigner = GlobalWindows.create();
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, GlobalWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Count.<GlobalWindow>atLeast(4)).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +4
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +8
+
+		Assert.assertEquals(4 + 8, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testCountTriggerGlobalOnce() throws Exception {
+		final int ALLOWED_LATENESS = 100;
+
+		GlobalWindows windowAssigner = GlobalWindows.create();
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, GlobalWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Count.<GlobalWindow>atLeast(4).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// +4
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);		// no firing
+
+		Assert.assertEquals(4, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerTumblingOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow().accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2000);							// ON_TIME					+6
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);	// + 7 late firing
+
+		testHarness.processWatermark(2200);
+
+		// dropped
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		Assert.assertEquals(testHarness.getOutput().size(), 6 + 7 + 3);	// +3 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithEarlyCountTumblingOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow().withEarlyTrigger(Count.<TimeWindow>atLeast(2)).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);	//	+2 early firing
+
+		Assert.assertEquals(2 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);	// no firing (ONCE)
+
+		Assert.assertEquals(2 + 1, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);	// no firing (ONCE)
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processWatermark(2000);							// firing ON_TIME
+
+		Assert.assertEquals(2 + 6 + 2, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);	// +7 late firing
+
+		testHarness.processWatermark(2200);    // this will be added but there is no late trigger specified so no firing
+
+		// dropped
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		Assert.assertEquals(2 + 6 + 7 + 3, testHarness.getOutput().size());	// +3 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeTriggerWithLateCountTumblingOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+		final int ALLOWED_LATENESS = 100;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow().withLateTrigger(Count.<TimeWindow>atLeast(2)).accumulating(),
+			ALLOWED_LATENESS);
+
+		testHarness.open();
+
+		testHarness.processWatermark(1000);
+
+		Assert.assertEquals(1, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// if we remove the following watermark, then we will have different
+		// results because we do not checkpoint the watermark and the window
+		// has not fired yet, to set the hasFiredOnTime flag.
+
+		testHarness.processWatermark(2000);							// no firing ON_TIME because no elements
+
+		Assert.assertEquals(2, testHarness.getOutput().size());		// for watermarks
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);	// +1 ON_TIME firing (although it is late)
+
+		Assert.assertEquals(1 + 2, testHarness.getOutput().size());	// for watermarks
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1000);	// +2 late firing
+
+		Assert.assertEquals(1 + 3 + 2, testHarness.getOutput().size());
+
+		// added as late
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);	// no firing because we are at Repeat.ONCE
+
+		Assert.assertEquals(1 + 3 + 2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		testHarness.processWatermark(2200);    // this will be added but there is no late trigger specified so no firing
+
+		// dropped
+		testHarness.processElement(new Tuple2<>("key1", 1), 1980);
+
+		Assert.assertEquals(1 + 3 + 3, testHarness.getOutput().size());	// +3 for the watermarks
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAnyOfTriggerWithTumblingWindowsOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Any.of(Count.<TimeWindow>atLeast(4),
+				ProcessingTime.<TimeWindow>afterFirstElement(Time.milliseconds(100))).accumulating(),
+			0);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(10);
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+
+		testHarness.setProcessingTime(110);							// +2 processing time
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);	// no firing
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +6 count_4 fires (accumulating)
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.setProcessingTime(250);							// no firing, state has been cleared
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.setProcessingTime(350);							// +9, processing time
+
+		Assert.assertEquals(2, testHarness.getOutput().size());
+
+		testHarness.processWatermark(2000);							// the cleanup timer
+
+		// check the we have processed all the timers
+		// (we do not have separate cleanup timer because lateness=0)
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testEventTimeWithEarlyOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			EventTime.<TimeWindow>afterEndOfWindow().withEarlyTrigger(Count.<TimeWindow>atLeast(3)).discarding(),
+			0);
+
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// no firing (ONCE)
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// no firing
+
+		testHarness.processWatermark(1999);							// +2 key1 +6 key2 fires on_time
+
+		Assert.assertEquals(3 + 2 + 6 + 1, testHarness.getOutput().size());// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testAnyOfEventTimeWithEarlyOnce() 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 = createTestHarness(
+			windowAssigner,
+			Any.of(EventTime.<TimeWindow>afterEndOfWindow().withEarlyTrigger(Count.<TimeWindow>atLeast(3))).discarding(),
+			0);
+
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// no firing (ONCE)
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// no firing
+
+		testHarness.processWatermark(1999);							// +2 key1 fires on_time
+
+		Assert.assertEquals(3 + 2 + 1, testHarness.getOutput().size());// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testForeverAnyOfEventTimeWithEarlyOnce() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Any.of(EventTime.<TimeWindow>afterEndOfWindow().withEarlyTrigger(Count.<TimeWindow>atLeast(3)))).discarding(),
+			0);
+
+		// this will be translated to this: Repeat.Forever(Any.of [EventTimeTrigger.afterEndOfWindow().withEarlyTrigger(Repeat.Once(CountTrigger(3)))]).discarding()
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(6, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(9, testHarness.getOutput().size());
+
+		testHarness.processWatermark(1999);							// +2 key1 +0 key2 fires on_time
+
+		Assert.assertEquals(9 + 2 + 1, testHarness.getOutput().size());// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+
+	@Test
+	public void testForeverAnyOfEventTimeWithEarlyForever() throws Exception {
+		final int WINDOW_SIZE = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = createTestHarness(
+			windowAssigner,
+			Repeat.Forever(Any.of(EventTime.<TimeWindow>afterEndOfWindow().withEarlyTrigger(Repeat.Forever(Count.<TimeWindow>atLeast(3))))).discarding(),
+			0);
+
+		// this will be translated to this: Repeat.Forever(Any.of [EventTimeTrigger.afterEndOfWindow().withEarlyTrigger(Repeat.Forever(CountTrigger(3)))]).discarding()
+		testHarness.open();
+
+		// do a snapshot, close and restore again
+		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		// normal element
+		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1001);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1002);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(3, testHarness.getOutput().size());
+
+		// do a snapshot, close and restore again
+		snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.close();
+		testHarness.restore(snapshot);
+		testHarness.open();
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1003);
+
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+		testHarness.processElement(new Tuple2<>("key1", 1), 1004);
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(6, testHarness.getOutput().size());
+
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);
+		testHarness.processElement(new Tuple2<>("key2", 1), 1005);	// +3 count_3 fires	key2
+
+		Assert.assertEquals(9, testHarness.getOutput().size());
+
+		testHarness.processWatermark(1999);							// +2 key1 +6 key2 fires on_time
+
+		Assert.assertEquals(9 + 2 + 1, testHarness.getOutput().size());// +1 for the watermark
+
+		// check the we have processed all the timers
+		Assert.assertEquals(new Tuple2<>(0, 0), testHarness.getOperator().getNumberOfTimers());
+
+		testHarness.close();
+		testHarness.dispose();
+	}
+}
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 fd73bcc5843..49e8a4a0f9c 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -24,9 +24,12 @@
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
@@ -89,6 +92,48 @@
 	// For counting if close() is called the correct number of times on the SumReducer
 	private static AtomicInteger closeCalled = new AtomicInteger(0);
 
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSlidingEventTimeWindowsApply() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3;
+		final int WINDOW_SLIDE = 1;
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createApplyWindowOperator(
+			SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
+
+		testHarness.open();
+
+		testSlidingEventTimeWindows(testHarness);
+
+		testHarness.close();
+
+		// we close once in the rest...
+		Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSlidingEventTimeWindowsReduce() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3;
+		final int WINDOW_SLIDE = 1;
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
+
+		testHarness.setup();
+		testHarness.open();
+
+		testSlidingEventTimeWindows(testHarness);
+
+		testHarness.close();
+	}
+
 	private void testSlidingEventTimeWindows(OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness) throws Exception {
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
@@ -157,78 +202,99 @@ private void testSlidingEventTimeWindows(OneInputStreamOperatorTestHarness<Tuple
 	}
 
 	@Test
-	@SuppressWarnings("unchecked")
-	public void testSlidingEventTimeWindowsReduce() throws Exception {
-		closeCalled.set(0);
-
+	public void testProcessingTimeSlidingWindowsReduce() throws Throwable {
 		final int WINDOW_SIZE = 3;
 		final int WINDOW_SLIDE = 1;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
 
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			SlidingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), testTimeProvider, ProcessingTimeTrigger.create(), 0);
 
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0);
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
 
-		operator.setInputType(inputType, new ExecutionConfig());
+		// timestamp is ignored in processing time
+		testTimeProvider.setCurrentTime(3);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		testTimeProvider.setCurrentTime(1000);
 
-		testHarness.setup();
-		testHarness.open();
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 999));
 
-		testSlidingEventTimeWindows(testHarness);
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
+
+		testTimeProvider.setCurrentTime(2000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
+
+		testTimeProvider.setCurrentTime(3000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
+
+		testTimeProvider.setCurrentTime(7000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 3999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 5), 3999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 5), 4999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
 
 		testHarness.close();
 	}
 
 	@Test
 	@SuppressWarnings("unchecked")
-	public void testSlidingEventTimeWindowsApply() throws Exception {
+	public void testTumblingEventTimeWindowsApply() throws Exception {
 		closeCalled.set(0);
 
 		final int WINDOW_SIZE = 3;
-		final int WINDOW_SLIDE = 1;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createApplyWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
 
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
+		testHarness.open();
 
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				EventTimeTrigger.create(),
-				0);
+		testTumblingEventTimeWindows(testHarness);
 
-		operator.setInputType(inputType, new ExecutionConfig());
+		testHarness.close();
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		// we close once in the rest...
+		Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testTumblingEventTimeWindowsReduce() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3;
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
 
 		testHarness.open();
 
-		testSlidingEventTimeWindows(testHarness);
+		testTumblingEventTimeWindows(testHarness);
 
 		testHarness.close();
-
-		// we close once in the rest...
-		Assert.assertEquals("Close was not called.", 2, closeCalled.get());
 	}
 
 	private void testTumblingEventTimeWindows(OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness) throws Exception {
@@ -295,75 +361,46 @@ private void testTumblingEventTimeWindows(OneInputStreamOperatorTestHarness<Tupl
 	}
 
 	@Test
-	@SuppressWarnings("unchecked")
-	public void testTumblingEventTimeWindowsReduce() throws Exception {
-		closeCalled.set(0);
-
+	public void testProcessingTimeTumblingWindowsReduce() throws Throwable {
 		final int WINDOW_SIZE = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0);
+		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), testTimeProvider, ProcessingTimeTrigger.create(), 0);
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
 		testHarness.open();
 
-		testTumblingEventTimeWindows(testHarness);
-
-		testHarness.close();
-	}
+		testTimeProvider.setCurrentTime(3);
 
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testTumblingEventTimeWindowsApply() throws Exception {
-		closeCalled.set(0);
+		// timestamp is ignored in processing time
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
 
-		final int WINDOW_SIZE = 3;
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		testTimeProvider.setCurrentTime(5000);
 
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
 
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				EventTimeTrigger.create(),
-				0);
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		testTimeProvider.setCurrentTime(7000);
 
-		testHarness.open();
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
 
-		testTumblingEventTimeWindows(testHarness);
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
 
 		testHarness.close();
-
-		// we close once in the rest...
-		Assert.assertEquals("Close was not called.", 2, closeCalled.get());
 	}
 
 	@Test
@@ -373,25 +410,8 @@ public void testSessionWindows() throws Exception {
 
 		final int SESSION_SIZE = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				EventTimeTrigger.create(),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+			createApplyWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)), EventTimeTrigger.create(), 0);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -444,28 +464,10 @@ public void testSessionWindows() throws Exception {
 	@SuppressWarnings("unchecked")
 	public void testReduceSessionWindows() throws Exception {
 		closeCalled.set(0);
-
 		final int SESSION_SIZE = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>(
-				"window-contents", new SumReducer(), inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				EventTimeTrigger.create(),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness = createApplyWindowOperatorWithTuple3(
+			EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)), EventTimeTrigger.create(), 0);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -514,7 +516,6 @@ public void testReduceSessionWindows() throws Exception {
 
 	/**
 	 * This tests whether merging works correctly with the CountTrigger.
-	 * @throws Exception
 	 */
 	@Test
 	@SuppressWarnings("unchecked")
@@ -523,26 +524,9 @@ public void testSessionWindowsWithCountTrigger() throws Exception {
 
 		final int SESSION_SIZE = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-		
+			createApplyWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)), PurgingTrigger.of(CountTrigger.<TimeWindow>of(4)), 0);
+
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
 		testHarness.open();
@@ -574,7 +558,7 @@ public void testSessionWindowsWithCountTrigger() throws Exception {
 
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
 
-		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
+		// add an element that merges the two "key1" sessions, they should now have count 6, and therefore fire
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
 
 		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
@@ -616,8 +600,6 @@ public String getKey(String value) throws Exception {
 
 	}
 
-	@Test
-	@SuppressWarnings("unchecked")
 	/**
 	 * This tests a custom Session window assigner that assigns some elements to "point windows",
 	 * windows that have the same timestamp for start and end.
@@ -625,29 +607,14 @@ public String getKey(String value) throws Exception {
 	 * <p> In this test, elements that have 33 as the second tuple field will be put into a point
 	 * window.
 	 */
+	@Test
+	@SuppressWarnings("unchecked")
 	public void testPointSessions() throws Exception {
 		closeCalled.set(0);
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				new PointSessionWindows(3000),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				EventTimeTrigger.create(),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-		
+			createApplyWindowOperatorWithTuple3(new PointSessionWindows(3000), EventTimeTrigger.create(), 0);
+
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
 		testHarness.open();
@@ -687,26 +654,9 @@ public void testContinuousWatermarkTrigger() throws Exception {
 
 		final int WINDOW_SIZE = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new WindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()),
-				ContinuousEventTimeTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperator(GlobalWindows.create(), new GlobalWindow.Serializer(),
+				ContinuousEventTimeTrigger.<GlobalWindow>of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), 0);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -777,27 +727,9 @@ public void testCountTrigger() throws Exception {
 
 		final int WINDOW_SIZE = 4;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new WindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()),
-				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)),
-				0);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(
-				"Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperator(GlobalWindows.create(), new GlobalWindow.Serializer(),
+				PurgingTrigger.of(CountTrigger.<GlobalWindow>of(WINDOW_SIZE)), 0);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -823,24 +755,8 @@ public void testCountTrigger() throws Exception {
 
 		ConcurrentLinkedQueue<Object> outputBeforeClose = testHarness.getOutput();
 
-		stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		operator = new WindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()),
-				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)),
-				0);
-
-		testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(
-				"Tuple2<String, Integer>"), new ExecutionConfig());
+		testHarness = createReducingWindowOperator(GlobalWindows.create(), new GlobalWindow.Serializer(),
+			PurgingTrigger.of(CountTrigger.<GlobalWindow>of(WINDOW_SIZE)), 0);
 
 		testHarness.setup();
 		testHarness.restore(snapshot);
@@ -861,9 +777,6 @@ public void testCountTrigger() throws Exception {
 		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 4), Long.MAX_VALUE));
 		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
 
-		System.out.println("BEFORE GOT: " + outputBeforeClose);
-		System.out.println("GOT: " + testHarness.getOutput());
-
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, Iterables.concat(outputBeforeClose, testHarness.getOutput()), new Tuple2ResultSortComparator());
 
 		testHarness.close();
@@ -871,36 +784,22 @@ public void testCountTrigger() throws Exception {
 
 	@Test
 	public void testRestoreAndSnapshotAreInSync() throws Exception {
-
 		final int WINDOW_SIZE = 3;
 		final int WINDOW_SLIDE = 1;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0);
-
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		WindowOperator.Timer<String, TimeWindow> timer1 = new WindowOperator.Timer<>(1L, "key1", new TimeWindow(1L, 2L));
 		WindowOperator.Timer<String, TimeWindow> timer2 = new WindowOperator.Timer<>(3L, "key1", new TimeWindow(1L, 2L));
 		WindowOperator.Timer<String, TimeWindow> timer3 = new WindowOperator.Timer<>(2L, "key1", new TimeWindow(1L, 2L));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
+			(WindowOperator) testHarness.getOperator();
+
 		operator.processingTimeTimers.add(timer1);
 		operator.processingTimeTimers.add(timer2);
 		operator.processingTimeTimers.add(timer3);
@@ -912,23 +811,14 @@ public void testRestoreAndSnapshotAreInSync() throws Exception {
 		operator.processingTimeTimerTimestamps.add(2L, 5);
 		operator.processingTimeTimerTimestamps.add(3L, 1);
 
-
 		StreamStateHandle snapshot = testHarness.snapshot(0, 0);
 
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> otherOperator = new WindowOperator<>(
-				SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> otherTestHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(otherOperator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> otherTestHarness = createReducingWindowOperator(
+			SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), 0);
 
-		otherOperator.setInputType(inputType, new ExecutionConfig());
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> otherOperator =
+			(WindowOperator) otherTestHarness.getOperator();
 
 		otherTestHarness.setup();
 		otherTestHarness.restore(snapshot);
@@ -940,164 +830,16 @@ public void testRestoreAndSnapshotAreInSync() throws Exception {
 	}
 
 	@Test
-	public void testProcessingTimeTumblingWindows() throws Throwable {
-		final int WINDOW_SIZE = 3;
+	public void testProcessingTimeSessionWindows() throws Throwable {
+		final int WINDOW_GAP = 3;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
 
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			ProcessingTimeSessionWindows.withGap(Time.of(WINDOW_GAP, TimeUnit.SECONDS)), new TimeWindow.Serializer(),
+			testTimeProvider, ProcessingTimeTrigger.create(), 0);
 
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(), 0);
-
-		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		testTimeProvider.setCurrentTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testTimeProvider.setCurrentTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testTimeProvider.setCurrentTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testProcessingTimeSlidingWindows() throws Throwable {
-		final int WINDOW_SIZE = 3;
-		final int WINDOW_SLIDE = 1;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(), 0);
-
-		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		// timestamp is ignored in processing time
-		testTimeProvider.setCurrentTime(3);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-
-		testTimeProvider.setCurrentTime(1000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-
-		testTimeProvider.setCurrentTime(2000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 1999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
-
-		testTimeProvider.setCurrentTime(3000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), Long.MAX_VALUE));
-
-		testTimeProvider.setCurrentTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 3999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 5), 3999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 5), 4999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testProcessingTimeSessionWindows() throws Throwable {
-		final int WINDOW_GAP = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				ProcessingTimeSessionWindows.withGap(Time.of(WINDOW_GAP, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(), 0);
-
-		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
 		testHarness.open();
 
@@ -1137,32 +879,17 @@ public void testProcessingTimeSessionWindows() throws Throwable {
 		testHarness.close();
 	}
 
+	//			LATENESS TESTING
+
 	@Test
 	public void testLateness() throws Exception {
 		final int WINDOW_SIZE = 2;
 		final long LATENESS = 500;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(),
+			PurgingTrigger.of(EventTimeTrigger.create()), LATENESS);
 
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				PurgingTrigger.of(EventTimeTrigger.create()),
-				LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 		
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1196,101 +923,14 @@ public void testLateness() throws Exception {
 		testHarness.close();
 	}
 
-	@Test
-	public void testCleanupTimeOverflow() throws Exception {
-		final int WINDOW_SIZE = 1000;
-		final long LATENESS = 2000;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
-
-		final WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-					windowAssigner,
-					new TimeWindow.Serializer(),
-					new TupleKeySelector(),
-					BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-					stateDesc,
-					new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-					EventTimeTrigger.create(),
-					LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
-		testHarness.open();
-
-		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
-
-		long timestamp = Long.MAX_VALUE - 1750;
-		Collection<TimeWindow> windows = windowAssigner.assignWindows(new Tuple2<>("key2", 1), timestamp, new WindowAssigner.WindowAssignerContext() {
-			@Override
-			public long getCurrentProcessingTime() {
-				return operator.windowAssignerContext.getCurrentProcessingTime();
-			}
-		});
-		TimeWindow window = Iterables.getOnlyElement(windows);
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), timestamp));
-
-		// the garbage collection timer would wrap-around
-		Assert.assertTrue(window.maxTimestamp() + LATENESS < window.maxTimestamp());
-
-		// and it would prematurely fire with watermark (Long.MAX_VALUE - 1500)
-		Assert.assertTrue(window.maxTimestamp() + LATENESS < Long.MAX_VALUE - 1500);
-
-		// if we don't correctly prevent wrap-around in the garbage collection
-		// timers this watermark will clean our window state for the just-added
-		// element/window
-		testHarness.processWatermark(new Watermark(Long.MAX_VALUE - 1500));
-
-		// this watermark is before the end timestamp of our only window
-		Assert.assertTrue(Long.MAX_VALUE - 1500 < window.maxTimestamp());
-		Assert.assertTrue(window.maxTimestamp() < Long.MAX_VALUE);
-
-		// push in a watermark that will trigger computation of our window
-		testHarness.processWatermark(new Watermark(window.maxTimestamp()));
-
-		expected.add(new Watermark(Long.MAX_VALUE - 1500));
-		expected.add(new StreamRecord<>(new Tuple2<>("key2", 1), window.maxTimestamp()));
-		expected.add(new Watermark(window.maxTimestamp()));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
 	@Test
 	public void testDropDueToLatenessTumbling() throws Exception {
 		final int WINDOW_SIZE = 2;
 		final long LATENESS = 0;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
 
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1331,27 +971,10 @@ public void testDropDueToLatenessSliding() throws Exception {
 		final int WINDOW_SLIDE = 1;
 		final long LATENESS = 0;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-				SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			SlidingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 		
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1407,27 +1030,9 @@ public void testDropDueToLatenessSessionZeroLatenessPurgingTrigger() throws Exce
 		final int GAP_SIZE = 3;
 		final long LATENESS = 0;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				PurgingTrigger.of(EventTimeTrigger.create()),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+				PurgingTrigger.of(EventTimeTrigger.create()), LATENESS);
 
 		testHarness.open();
 		
@@ -1499,27 +1104,9 @@ public void testDropDueToLatenessSessionZeroLateness() throws Exception {
 		final int GAP_SIZE = 3;
 		final long LATENESS = 0;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+				EventTimeTrigger.create(), LATENESS);
 
 		testHarness.open();
 		
@@ -1578,34 +1165,12 @@ public void testDropDueToLatenessSessionZeroLateness() throws Exception {
 
 	@Test
 	public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exception {
-
-		// this has the same output as testDropDueToLatenessSessionZeroLateness() because
-		// the allowed lateness is too small to make a difference
-
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				PurgingTrigger.of(EventTimeTrigger.create()),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+				PurgingTrigger.of(EventTimeTrigger.create()), LATENESS);
 
 		testHarness.open();
 
@@ -1621,8 +1186,7 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce
 
 		expected.add(new Watermark(4998));
 
-		// this will not be dropped because the session we're adding two has maxTimestamp
-		// after the current watermark
+		// this will not be dropped because the session is still active (allowed lateness not expired)
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 4500));
 
 		// new session
@@ -1638,6 +1202,9 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-5", 1000L, 11500L), 11499));
 		expected.add(new Watermark(11501));
 
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
+
 		// new session
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 11600));
 		testHarness.processWatermark(new Watermark(14600));
@@ -1645,20 +1212,24 @@ public void testDropDueToLatenessSessionWithLatenessPurgingTrigger() throws Exce
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599));
 		expected.add(new Watermark(14600));
 
-		// dropped as late
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
+
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000));
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 14600L), 14599));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
 
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
-		testHarness.processWatermark(new Watermark(20000));
+		testHarness.processWatermark(new Watermark(17509)); // this is the cleanup time
 
-		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 14500L, 17500L), 17499));
-		expected.add(new Watermark(20000));
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 10000L, 17500L), 17499));
+		expected.add(new Watermark(17509));
 
-		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());
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(0, 0)));	// all timers are processed
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
 		testHarness.close();
 	}
 
@@ -1671,27 +1242,8 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception {
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)), EventTimeTrigger.create(), LATENESS);
 
 		testHarness.open();
 		
@@ -1736,9 +1288,7 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception {
 		// be dropped as late
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 10000));
 
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
-
-		// adding ("key2", 1) extended the session to (10000-146000) for which
+		// adding ("key2", 1) extended the session to (10000-14600) for which
 		// maxTimestamp <= currentWatermark. Therefore, we immediately get a firing
 		// with the current version of EventTimeTrigger/EventTimeTriggerAccum
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-2", 10000L, 14600L), 14599));
@@ -1746,6 +1296,7 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception {
 		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
 
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
 		testHarness.processWatermark(new Watermark(20000));
 
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-3", 10000L, 17500L), 17499));
@@ -1762,31 +1313,12 @@ public void testDropDueToLatenessSessionWithLateness() throws Exception {
 
 	@Test
 	public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws Exception {
-
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10000;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				PurgingTrigger.of(EventTimeTrigger.create()),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+				PurgingTrigger.of(EventTimeTrigger.create()), LATENESS);
 
 		testHarness.open();
 		
@@ -1796,24 +1328,30 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws
 		testHarness.processWatermark(new Watermark(1999));
 
 		expected.add(new Watermark(1999));
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(2, 0)));	// end-of-window 3999 + cleanup at 13999
 
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 2000));
 		testHarness.processWatermark(new Watermark(4998));
 
 		expected.add(new Watermark(4998));
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(3, 0)));	// - 3999 + end-of-window 4999 + cleanup at 14999
 
-		// this will not be dropped because the session we're adding two has maxTimestamp
-		// after the current watermark
+		// this will not be dropped because the window is still active
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 4500));
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(4, 0)));	// - 4999 + end-of-window 7499 + cleanup at 17499
 
 		// new session
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 8500));
-		testHarness.processWatermark(new Watermark(7400));
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(6, 0)));	// end-of-window 11499 + cleanup at 21499
 
+		testHarness.processWatermark(new Watermark(7400));
 		expected.add(new Watermark(7400));
 
 		// this will merge the two sessions into one
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(6, 0)));	// nothing here
+
+		// this will fire the new merged window
 		testHarness.processWatermark(new Watermark(11501));
 
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-5", 1000L, 11500L), 11499));
@@ -1826,25 +1364,27 @@ public void testDropDueToLatenessSessionWithHugeLatenessPurgingTrigger() throws
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-1", 11600L, 14600L), 14599));
 		expected.add(new Watermark(14600));
 
+		// this will merge the two previous sessions (although they have fired,
+		// they are not yet expired) and it will fire the window again
 		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());
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
 
 		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));
-
-		expected.add(new Watermark(100000));
+		testHarness.processWatermark(new Watermark(28000));
+		expected.add(new Watermark(28000));
 
-		actual = testHarness.getOutput();
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(0, 0)));	// all timers are processed
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected,
+			testHarness.getOutput(), new Tuple3ResultSortComparator());
 		testHarness.close();
 	}
 
@@ -1853,27 +1393,8 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception {
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10000;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)), EventTimeTrigger.create(), LATENESS);
 
 		testHarness.open();
 
@@ -1919,48 +1440,82 @@ public void testDropDueToLatenessSessionWithHugeLateness() throws Exception {
 		// so we get an immediate firing
 		expected.add(new StreamRecord<>(new Tuple3<>("key2-7", 1000L, 14600L), 14599));
 
-		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
+		ConcurrentLinkedQueue<Object> actual = testHarness.getOutput();
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
+		testHarness.processWatermark(new Watermark(20000));
+
+		expected.add(new StreamRecord<>(new Tuple3<>("key2-8", 1000L, 17500L), 17499));
+		expected.add(new Watermark(20000));
+
+		testHarness.processWatermark(new Watermark(100000));
+		expected.add(new Watermark(100000));
+
+		actual = testHarness.getOutput();
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
+		testHarness.close();
+	}
+
+	@Test
+	public void testCleanupTimeOverflow() throws Exception {
+		final int WINDOW_SIZE = 1000;
+		final long LATENESS = 2000;
+
+		TumblingEventTimeWindows windowAssigner = TumblingEventTimeWindows.of(Time.milliseconds(WINDOW_SIZE));
+
+		final OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			windowAssigner, new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
+
+		testHarness.open();
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		long timestamp = Long.MAX_VALUE - 1750;
+		Collection<TimeWindow> windows = windowAssigner.assignWindows(new Tuple2<>("key2", 1), timestamp, new WindowAssigner.WindowAssignerContext() {
+			@Override
+			public long getCurrentProcessingTime() {
+				return ((WindowOperator) testHarness.getOperator()).windowAssignerContext.getCurrentProcessingTime();
+			}
+		});
+		TimeWindow window = Iterables.getOnlyElement(windows);
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), timestamp));
+
+		// the garbage collection timer would wrap-around
+		Assert.assertTrue(window.maxTimestamp() + LATENESS < window.maxTimestamp());
+
+		// and it would prematurely fire with watermark (Long.MAX_VALUE - 1500)
+		Assert.assertTrue(window.maxTimestamp() + LATENESS < Long.MAX_VALUE - 1500);
+
+		// if we don't correctly prevent wrap-around in the garbage collection
+		// timers this watermark will clean our window state for the just-added
+		// element/window
+		testHarness.processWatermark(new Watermark(Long.MAX_VALUE - 1500));
 
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 14500));
-		testHarness.processWatermark(new Watermark(20000));
+		// this watermark is before the end timestamp of our only window
+		Assert.assertTrue(Long.MAX_VALUE - 1500 < window.maxTimestamp());
+		Assert.assertTrue(window.maxTimestamp() < Long.MAX_VALUE);
 
-		expected.add(new StreamRecord<>(new Tuple3<>("key2-8", 1000L, 17500L), 17499));
-		expected.add(new Watermark(20000));
+		// push in a watermark that will trigger computation of our window
+		testHarness.processWatermark(new Watermark(window.maxTimestamp()));
 
-		testHarness.processWatermark(new Watermark(100000));
-		expected.add(new Watermark(100000));
+		expected.add(new Watermark(Long.MAX_VALUE - 1500));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 1), window.maxTimestamp()));
+		expected.add(new Watermark(window.maxTimestamp()));
 
-		actual = testHarness.getOutput();
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, actual, new Tuple3ResultSortComparator());
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
 		testHarness.close();
 	}
 
 	@Test
-	public void testCleanupTimerWithEmptyListStateForTumblingWindows2() throws Exception {
+	public void testCleanupTimerWithEmptyListStateForTumblingWindows() throws Exception {
 		final int WINDOW_SIZE = 2;
-		final long LATENESS = 100;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> windowStateDesc =
-			new ListStateDescriptor<>("window-contents", inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, String, TimeWindow> operator =
-			new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowStateDesc,
-				new InternalIterableWindowFunction<>(new PassThroughFunction2()),
-				new EventTimeTriggerAccumGC(LATENESS),
-				LATENESS);
+		final long LATENESS = 1;
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, String> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createApplyWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1969,53 +1524,43 @@ public void testCleanupTimerWithEmptyListStateForTumblingWindows2() throws Excep
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
 		testHarness.processWatermark(new Watermark(1599));
 		testHarness.processWatermark(new Watermark(1999));
-		testHarness.processWatermark(new Watermark(2100));
+		testHarness.processWatermark(new Watermark(2000));
 		testHarness.processWatermark(new Watermark(5000));
 
 		expected.add(new Watermark(1599));
-		expected.add(new StreamRecord<>("GOT: (key2,1)", 1999));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
 		expected.add(new Watermark(1999)); // here it fires and purges
-		expected.add(new Watermark(2100)); // here is the cleanup timer
+		expected.add(new Watermark(2000)); // here is the cleanup timer
 		expected.add(new Watermark(5000));
 
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
 		testHarness.close();
 	}
 
-	private class PassThroughFunction2 implements WindowFunction<Tuple2<String, Integer>, String, String, TimeWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void apply(String k, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<String> out) throws Exception {
-			out.collect("GOT: " + Joiner.on(",").join(input));
-		}
-	}
-
 	@Test
-	public void testCleanupTimerWithEmptyListStateForTumblingWindows() throws Exception {
+	public void testCleanupTimerWithEmptyListStateForTumblingWindows2() throws Exception {
 		final int WINDOW_SIZE = 2;
-		final long LATENESS = 1;
+		final long LATENESS = 100;
 
+		ExecutionConfig config = new ExecutionConfig();
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		ListStateDescriptor<Tuple2<String, Integer>> windowStateDesc = new ListStateDescriptor<>("window-contents", inputType.createSerializer(config));
 
-		ListStateDescriptor<Tuple2<String, Integer>> windowStateDesc =
-			new ListStateDescriptor<>("window-contents", inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator =
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, String, TimeWindow> operator =
 			new WindowOperator<>(
 				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
 				new TimeWindow.Serializer(),
 				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(config),
 				windowStateDesc,
-				new InternalIterableWindowFunction<>(new PassThroughFunction()),
-				EventTimeTrigger.create(),
+				new InternalIterableWindowFunction<>(new PassThroughFunction2()),
+				new EventTimeTriggerAccumGC(LATENESS),
 				LATENESS);
 
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, String> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
-		operator.setInputType(inputType, new ExecutionConfig());
+		operator.setInputType(inputType, config);
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2024,45 +1569,36 @@ public void testCleanupTimerWithEmptyListStateForTumblingWindows() throws Except
 		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
 		testHarness.processWatermark(new Watermark(1599));
 		testHarness.processWatermark(new Watermark(1999));
-		testHarness.processWatermark(new Watermark(2000));
+		testHarness.processWatermark(new Watermark(2100));
 		testHarness.processWatermark(new Watermark(5000));
 
 		expected.add(new Watermark(1599));
-		expected.add(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
+		expected.add(new StreamRecord<>("GOT: (key2,1)", 1999));
 		expected.add(new Watermark(1999)); // here it fires and purges
-		expected.add(new Watermark(2000)); // here is the cleanup timer
+		expected.add(new Watermark(2100)); // here is the cleanup timer
 		expected.add(new Watermark(5000));
 
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
 		testHarness.close();
 	}
 
+	private class PassThroughFunction2 implements WindowFunction<Tuple2<String, Integer>, String, String, TimeWindow> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void apply(String k, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<String> out) throws Exception {
+			out.collect("GOT: " + Joiner.on(",").join(input));
+		}
+	}
+
 	@Test
 	public void testCleanupTimerWithEmptyReduceStateForTumblingWindows() throws Exception {
 		final int WINDOW_SIZE = 2;
 		final long LATENESS = 1;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createReducingWindowOperator(
+			TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)), new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2089,8 +1625,8 @@ public void testCleanupTimerWithEmptyFoldingStateForTumblingWindows() throws Exc
 		final int WINDOW_SIZE = 2;
 		final long LATENESS = 1;
 
+		ExecutionConfig config = new ExecutionConfig();
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
 		FoldingStateDescriptor<Tuple2<String, Integer>, Tuple2<String, Integer>> windowStateDesc =
 			new FoldingStateDescriptor<>(
 				"window-contents",
@@ -2109,7 +1645,7 @@ public void testCleanupTimerWithEmptyFoldingStateForTumblingWindows() throws Exc
 				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
 				new TimeWindow.Serializer(),
 				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(config),
 				windowStateDesc,
 				new InternalSingleValueWindowFunction<>(new PassThroughFunction()),
 				EventTimeTrigger.create(),
@@ -2118,7 +1654,7 @@ public void testCleanupTimerWithEmptyFoldingStateForTumblingWindows() throws Exc
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
+		operator.setInputType(inputType, config);
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2145,26 +1681,10 @@ public void testCleanupTimerWithEmptyListStateForSessionWindows() throws Excepti
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> windowStateDesc =
-			new ListStateDescriptor<>("window-contents", inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowStateDesc,
-				new InternalIterableWindowFunction<>(new PassThroughFunction()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness = createApplyWindowOperator(
+			EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+			new TimeWindow.Serializer(), EventTimeTrigger.create(), LATENESS);
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2185,31 +1705,12 @@ public void testCleanupTimerWithEmptyListStateForSessionWindows() throws Excepti
 
 	@Test
 	public void testCleanupTimerWithEmptyReduceStateForSessionWindows() throws Exception {
-
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10;
 
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-			new SumReducer(),
-			inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
-			new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()),
-				EventTimeTrigger.create(),
-				LATENESS);
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			createReducingWindowOperatorWithTuple3(EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
+				EventTimeTrigger.create(), LATENESS);
 
 		testHarness.open();
 
@@ -2234,8 +1735,8 @@ public void testCleanupTimerWithEmptyFoldingStateForSessionWindows() throws Exce
 		final int GAP_SIZE = 3;
 		final long LATENESS = 10;
 
+		ExecutionConfig config = new ExecutionConfig();
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
 		FoldingStateDescriptor<Tuple2<String, Integer>, Tuple2<String, Integer>> windowStateDesc =
 			new FoldingStateDescriptor<>(
 				"window-contents",
@@ -2254,16 +1755,16 @@ public void testCleanupTimerWithEmptyFoldingStateForSessionWindows() throws Exce
 				EventTimeSessionWindows.withGap(Time.seconds(GAP_SIZE)),
 				new TimeWindow.Serializer(),
 				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(config),
 				windowStateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughFunction()),
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
 				EventTimeTrigger.create(),
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
+		operator.setInputType(inputType, config);
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2282,6 +1783,100 @@ public void testCleanupTimerWithEmptyFoldingStateForSessionWindows() throws Exce
 		testHarness.close();
 	}
 
+	@Test
+	public void testPurgingTriggerOnFirePropagation() throws Exception {
+		final int WINDOW_SIZE = 2;
+		final long LATENESS = 500;
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+			createReducingWindowOperator(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE)), new TimeWindow.Serializer(),
+				PurgingTrigger.of(new TestCountTrigger<TimeWindow>(4)), LATENESS);
+
+		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 500));
+		testHarness.processWatermark(new Watermark(1500));
+		expected.add(new Watermark(1500));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 4), 1999));
+
+		testHarness.processWatermark(new Watermark(2300));
+		expected.add(new Watermark(2300));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1300));
+		expected.add(new StreamRecord<>(new Tuple2<>("key2", 4), 1999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expected, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(2700));
+		expected.add(new Watermark(2700));
+
+		Assert.assertTrue(((WindowOperator) testHarness.getOperator()).getNumberOfTimers().equals(new Tuple2<>(0, 0)));
+		testHarness.close();
+	}
+
+	private class TestCountTrigger<W extends Window> extends Trigger<Object, W> {
+		private static final long serialVersionUID = 1L;
+
+		private final long maxCount;
+
+		private final ReducingStateDescriptor<Long> stateDesc =
+			new ReducingStateDescriptor<>("count", new Sum(), LongSerializer.INSTANCE);
+
+		private TestCountTrigger(long maxCount) {
+			this.maxCount = maxCount;
+		}
+
+		@Override
+		public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) throws Exception {
+			ReducingState<Long> count = ctx.getPartitionedState(stateDesc);
+			count.add(1L);
+			return (count.get() >= maxCount) ? TriggerResult.FIRE : TriggerResult.CONTINUE;
+		}
+
+		@Override
+		public TriggerResult onEventTime(long time, W window, TriggerContext ctx) {
+			return TriggerResult.CONTINUE;
+		}
+
+		@Override
+		public void onFire(W window, TriggerContext ctx) throws Exception {
+			ctx.getPartitionedState(stateDesc).clear();
+		}
+
+		@Override
+		public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) throws Exception {
+			return TriggerResult.CONTINUE;
+		}
+
+		@Override
+		public void clear(W window, TriggerContext ctx) throws Exception {
+			// do nothing, just for the testing
+		}
+
+		private class Sum implements ReduceFunction<Long> {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Long reduce(Long value1, Long value2) throws Exception {
+				return value1 + value2;
+			}
+
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  UDFs
 	// ------------------------------------------------------------------------
@@ -2307,7 +1902,7 @@ public void apply(String k, TimeWindow window, Iterable<Tuple2<String, Integer>>
 	}
 
 
-	public static class RichSumReducer<W extends Window> extends RichWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, W> {
+	private static class RichSumReducer<W extends Window> extends RichWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, W> {
 		private static final long serialVersionUID = 1L;
 
 		private boolean openCalled = false;
@@ -2345,7 +1940,7 @@ public void apply(String key,
 	}
 
 	@SuppressWarnings("unchecked")
-	private static class Tuple2ResultSortComparator implements Comparator<Object> {
+	static class Tuple2ResultSortComparator implements Comparator<Object> {
 		@Override
 		public int compare(Object o1, Object o2) {
 			if (o1 instanceof Watermark || o2 instanceof Watermark) {
@@ -2401,7 +1996,7 @@ public String getKey(Tuple2<String, Integer> value) throws Exception {
 		}
 	}
 
-	public static class SessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
+	private static class SessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -2418,7 +2013,7 @@ public void apply(String key,
 		}
 	}
 
-	public static class ReducedSessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
+	static class ReducedSessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -2432,8 +2027,7 @@ public void apply(String key,
 		}
 	}
 
-
-	public static class PointSessionWindows extends EventTimeSessionWindows {
+	private static class PointSessionWindows extends EventTimeSessionWindows {
 		private static final long serialVersionUID = 1L;
 
 
@@ -2459,16 +2053,12 @@ private PointSessionWindows(long sessionTimeout) {
 	 * purge the state of the fired window. This is to test the state
 	 * garbage collection mechanism.
 	 */
-	public class EventTimeTriggerAccumGC extends Trigger<Object, TimeWindow> {
+	private class EventTimeTriggerAccumGC extends Trigger<Object, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		private long cleanupTime;
 
-		private EventTimeTriggerAccumGC() {
-			cleanupTime = 0L;
-		}
-
-		public EventTimeTriggerAccumGC(long cleanupTime) {
+		private EventTimeTriggerAccumGC(long cleanupTime) {
 			this.cleanupTime = cleanupTime;
 		}
 
@@ -2529,12 +2119,14 @@ public void testEventTimeTumblingWindowsWithOffset() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			EventTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		// normal element
 		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
 		testHarness.processWatermark(1985);
@@ -2578,12 +2170,14 @@ public void testEventTimeSlidingWindowsWithOffset() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			EventTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		testHarness.processElement(new Tuple2<>("key2", 1), 333);
 		testHarness.processWatermark(6666);
 
@@ -2610,12 +2204,14 @@ public void testProcessingTimeTumblingWindowsWithOffset() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			ProcessingTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		testHarness.setProcessingTime(3);
 
 		// timestamp is ignored in processing time
@@ -2665,12 +2261,14 @@ public void testProcessingTimeSlidingWindowsWithOffset() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			ProcessingTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		testHarness.setProcessingTime(3);
 
 		// timestamp is ignored in processing time
@@ -2691,4 +2289,112 @@ public void testProcessingTimeSlidingWindowsWithOffset() throws Exception {
 
 		testHarness.close();
 	}
+
+	//			Helper methods
+
+	private <W extends Window> OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> createReducingWindowOperator(
+		WindowAssigner<Object, W> windowAssigner, TypeSerializer<W> windowSerializer, Trigger<Object, W> trigger, long lateness) {
+		return createReducingWindowOperator(windowAssigner, windowSerializer, null, trigger, lateness);
+	}
+
+	private <W extends Window> OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> createReducingWindowOperator(
+		WindowAssigner<Object, W> windowAssigner, TypeSerializer<W> windowSerializer,
+		TestTimeServiceProvider testTimeProvider, Trigger<Object, W> trigger, long lateness) {
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc =
+			new ReducingStateDescriptor<>("window-contents", new SumReducer(), inputType.createSerializer(config));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, W> operator =
+			new WindowOperator<>(windowAssigner, windowSerializer, keySelector, keyType.createSerializer(config), stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, W, Tuple2<String, Integer>>()), trigger, lateness);
+		operator.setInputType(inputType, config);
+
+		return testTimeProvider == null ? new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, keyType) :
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, testTimeProvider, keySelector, keyType);
+	}
+
+	private <W extends Window> OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> createApplyWindowOperator(
+		WindowAssigner<Object, W> windowAssigner, TypeSerializer<W> windowSerializer, Trigger<Object, W> trigger, long lateness) {
+		return createApplyWindowOperator(windowAssigner, windowSerializer, null, trigger, lateness);
+	}
+
+	private <W extends Window> OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> createApplyWindowOperator(
+		WindowAssigner<Object, W> windowAssigner, TypeSerializer<W> windowSerializer,
+		TestTimeServiceProvider testTimeProvider, Trigger<Object, W> trigger, long lateness) {
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents", inputType.createSerializer(config));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, W> operator =
+			new WindowOperator<>(windowAssigner, windowSerializer, keySelector, keyType.createSerializer(config), stateDesc,
+				new InternalIterableWindowFunction<>(new RichSumReducer<W>()), trigger, lateness);
+		operator.setInputType(inputType, config);
+
+		return testTimeProvider == null ?
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, keyType) :
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, testTimeProvider, keySelector, keyType);
+	}
+
+	private OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> createApplyWindowOperatorWithTuple3(
+		WindowAssigner<Object, TimeWindow> windowAssigner, Trigger<Object, TimeWindow> trigger, long lateness) {
+		return createApplyWindowOperatorWithTuple3(windowAssigner, null, trigger, lateness);
+	}
+
+	private OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> createApplyWindowOperatorWithTuple3(
+		WindowAssigner<Object, TimeWindow> windowAssigner, TestTimeServiceProvider testTimeProvider, Trigger<Object, TimeWindow> trigger, long lateness) {
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents", inputType.createSerializer(config));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator =
+			new WindowOperator<>(windowAssigner, new TimeWindow.Serializer(), keySelector, keyType.createSerializer(config), stateDesc,
+				new InternalIterableWindowFunction(new SessionWindowFunction()), trigger, lateness);
+		operator.setInputType(inputType, config);
+
+		return testTimeProvider == null ?
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, keyType) :
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, testTimeProvider, keySelector, keyType);
+	}
+
+	private OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> createReducingWindowOperatorWithTuple3(
+		WindowAssigner<Object, TimeWindow> windowAssigner, Trigger<Object, TimeWindow> trigger, long lateness) {
+		return createReducingWindowOperatorWithTuple3(windowAssigner, null, trigger, lateness);
+	}
+
+	private OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> createReducingWindowOperatorWithTuple3(
+		WindowAssigner<Object, TimeWindow> windowAssigner, TestTimeServiceProvider testTimeProvider, Trigger<Object, TimeWindow> trigger, long lateness) {
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ExecutionConfig config = new ExecutionConfig();
+		KeySelector<Tuple2<String, Integer>, String> keySelector = new TupleKeySelector();
+		TypeInformation<String> keyType = BasicTypeInfo.STRING_TYPE_INFO;
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc =
+			new ReducingStateDescriptor<>("window-contents", new SumReducer(), inputType.createSerializer(config));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Long, Long>, TimeWindow> operator =
+			new WindowOperator<>(windowAssigner, new TimeWindow.Serializer(), keySelector, keyType.createSerializer(config), stateDesc,
+				new InternalSingleValueWindowFunction<>(new ReducedSessionWindowFunction()), trigger, lateness);
+		operator.setInputType(inputType, config);
+
+		return testTimeProvider == null ? new KeyedOneInputStreamOperatorTestHarness<>(operator, keySelector, keyType) :
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, config, testTimeProvider, keySelector, keyType);
+	}
 }
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index 27071081641..a2cd44ca691 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
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
index 58a7897562d..8cc4ecc5b54 100644
--- 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -47,12 +47,14 @@ public void testEventTimeTumblingWindows() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			EventTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		// normal element
 		testHarness.processElement(new Tuple2<>("key2", 1), 1000);
 		testHarness.processWatermark(1985);
@@ -81,6 +83,7 @@ public void testEventTimeTumblingWindows() throws Exception {
 		testHarness.compareActualToExpectedOutput("Output is not correct");
 
 		testHarness.close();
+		testHarness.dispose();
 	}
 
 	@Test
@@ -94,12 +97,14 @@ public void testProcessingTimeTumblingWindows() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			ProcessingTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		testHarness.setProcessingTime(3);
 
 		// timestamp is ignored in processing time
@@ -133,6 +138,7 @@ public void testProcessingTimeTumblingWindows() throws Exception {
 		testHarness.compareActualToExpectedOutput("Output was not correct.");
 
 		testHarness.close();
+		testHarness.dispose();
 	}
 
 	@Test
@@ -147,12 +153,14 @@ public void testSnapshotingAndRecovery() throws Exception {
 		WindowingTestHarness<String, Tuple2<String, Integer>, TimeWindow> testHarness = new WindowingTestHarness<>(
 			new ExecutionConfig(),
 			windowAssigner,
+			new TupleKeySelector(),
 			BasicTypeInfo.STRING_TYPE_INFO,
 			inputType,
-			new TupleKeySelector(),
 			EventTimeTrigger.create(),
 			0);
 
+		testHarness.open();
+
 		// add elements out-of-order
 		testHarness.processElement(new Tuple2<>("key2", 1), 3999);
 		testHarness.processElement(new Tuple2<>("key2", 1), 3000);
@@ -178,6 +186,7 @@ public void testSnapshotingAndRecovery() throws Exception {
 		StreamStateHandle snapshot = testHarness.snapshot(0L, 0L);
 		testHarness.close();
 		testHarness.restore(snapshot);
+		testHarness.open();
 
 		testHarness.processWatermark(2999);
 
@@ -220,6 +229,8 @@ public void testSnapshotingAndRecovery() throws Exception {
 		testHarness.addExpectedWatermark(7999);
 
 		testHarness.compareActualToExpectedOutput("Output was not correct.");
+
+		testHarness.dispose();
 	}
 
 	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index acf046ad647..54f4e68a1a4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -39,7 +39,6 @@
 import org.apache.flink.streaming.runtime.tasks.AsynchronousException;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -161,6 +160,10 @@ public TimeServiceProvider answer(InvocationOnMock invocation) throws Throwable
 		}).when(mockTask).getTimerService();
 	}
 
+	public OneInputStreamOperator<IN, OUT> getOperator() {
+		return this.operator;
+	}
+
 	public void setStateBackend(AbstractStateBackend stateBackend) {
 		this.stateBackend = stateBackend;
 	}
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
index d47136c130b..b379c317267 100644
--- 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,6 +27,8 @@
 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.triggers.triggerdsl.DslTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTriggerRunner;
 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;
@@ -50,6 +52,8 @@
  */
 public class WindowingTestHarness<K, IN, W extends Window> {
 
+	private final WindowOperator<K, IN, Iterable<IN>, IN, W> operator;
+
 	private final TestTimeServiceProvider timeServiceProvider;
 
 	private final OneInputStreamOperatorTestHarness<IN, IN> testHarness;
@@ -60,17 +64,26 @@
 
 	public WindowingTestHarness(ExecutionConfig executionConfig,
 								WindowAssigner<? super IN, W> windowAssigner,
+								KeySelector<IN, K> keySelector,
 								TypeInformation<K> keyType,
 								TypeInformation<IN> inputType,
+								DslTrigger<? super IN, W> trigger,
+								long allowedLateness) {
+		this(executionConfig, windowAssigner, keySelector, keyType, inputType, new DslTriggerRunner<>(trigger), allowedLateness);
+	}
+
+	public WindowingTestHarness(ExecutionConfig executionConfig,
+								WindowAssigner<? super IN, W> windowAssigner,
 								KeySelector<IN, K> keySelector,
+								TypeInformation<K> keyType,
+								TypeInformation<IN> inputType,
 								Trigger<? super IN, ? super W> trigger,
 								long allowedLateness) {
 
 		ListStateDescriptor<IN> windowStateDesc =
 				new ListStateDescriptor<>("window-contents", inputType.createSerializer(executionConfig));
 
-		WindowOperator<K, IN, Iterable<IN>, IN, W> operator =
-			new WindowOperator<>(
+		operator = new WindowOperator<>(
 				windowAssigner,
 				windowAssigner.getWindowSerializer(executionConfig),
 				keySelector,
@@ -86,11 +99,14 @@ public WindowingTestHarness(ExecutionConfig executionConfig,
 		testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, executionConfig, timeServiceProvider, keySelector, keyType);
 	}
 
+	public WindowOperator getOperator() {
+		return this.operator;
+	}
+
 	/**
 	 * Simulates the processing of a new incoming element.
 	 */
 	public void processElement(IN element, long timestamp) throws Exception {
-		openOperator();
 		testHarness.processElement(new StreamRecord<>(element, timestamp));
 	}
 
@@ -98,7 +114,6 @@ public void processElement(IN element, long timestamp) throws Exception {
 	 * Simulates the processing of a new incoming watermark.
 	 */
 	public void processWatermark(long timestamp) throws Exception {
-		openOperator();
 		testHarness.processWatermark(new Watermark(timestamp));
 	}
 
@@ -107,7 +122,6 @@ public void processWatermark(long timestamp) throws Exception {
 	 * This is useful when working on processing time.
 	 */
 	public void setProcessingTime(long timestamp) throws Exception {
-		openOperator();
 		timeServiceProvider.setCurrentTime(timestamp);
 	}
 
@@ -130,6 +144,10 @@ public void close() throws Exception {
 		}
 	}
 
+	public void dispose() throws Exception {
+		close();
+	}
+
 	/**
 	 * Adds a watermark to the expected output.
 	 *
@@ -180,10 +198,9 @@ public void restore(StreamStateHandle stateHandle) throws Exception {
 
 		testHarness.setup();
 		testHarness.restore(stateHandle);
-		openOperator();
 	}
 
-	private void openOperator() throws Exception {
+	public void open() throws Exception {
 		if (!isOpen) {
 			testHarness.open();
 			isOpen = true;
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
index 47c13c9403f..9543a0400bd 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -18,17 +18,18 @@
 
 package org.apache.flink.streaming.api.scala
 
-import org.apache.flink.annotation.{PublicEvolving, Public}
+import org.apache.flink.annotation.{Public, PublicEvolving}
 import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream}
 import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
 import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
 import org.apache.flink.streaming.api.scala.function.AllWindowFunction
-import org.apache.flink.streaming.api.scala.function.util.{ScalaAllWindowFunction, ScalaAllWindowFunctionWrapper, ScalaReduceFunction, ScalaFoldFunction}
+import org.apache.flink.streaming.api.scala.function.util.{ScalaAllWindowFunction, ScalaAllWindowFunctionWrapper, ScalaFoldFunction, ScalaReduceFunction}
 import org.apache.flink.streaming.api.windowing.evictors.Evictor
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTrigger
 import org.apache.flink.streaming.api.windowing.windows.Window
 import org.apache.flink.util.Collector
 
@@ -79,6 +80,15 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
     this
   }
 
+  /**
+    * Sets the [[Trigger]] that should be used to trigger window emission.
+    */
+  @PublicEvolving
+  def trigger(trigger: DslTrigger[_ >: T, _ >: W]): AllWindowedStream[T, W] = {
+    javaStream.trigger(trigger)
+    this
+  }
+
   /**
    * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
    *
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
index 6a10ff63acf..0fe0db2d00f 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.api.scala
 
-import org.apache.flink.annotation.{PublicEvolving, Public}
+import org.apache.flink.annotation.{Public, PublicEvolving}
 import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream}
@@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.scala.function.util.{ScalaFoldFunction, Sc
 import org.apache.flink.streaming.api.windowing.evictors.Evictor
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.triggers.triggerdsl.DslTrigger
 import org.apache.flink.streaming.api.windowing.windows.Window
 import org.apache.flink.util.Collector
 
@@ -82,6 +83,15 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
     this
   }
 
+  /**
+    * Sets the [[Trigger]] that should be used to trigger window emission.
+    */
+  @PublicEvolving
+  def trigger(trigger: DslTrigger[_ >: T, _ >: W]): WindowedStream[T, K, W] = {
+    javaStream.trigger(trigger)
+    this
+  }
+
   /**
    * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
    *
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
index 6d90239532d..e15f0e49c2c 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
@@ -102,7 +102,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .windowAll(SlidingEventTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .reduce(reducer)
 
     val transform1 = window1.javaStream.getTransformation
@@ -120,7 +120,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .windowAll(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
       def apply(
                     window: TimeWindow,
@@ -170,7 +170,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
 
     val window2 = source
       .windowAll(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .evictor(CountEvictor.of(1000))
       .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
       def apply(
@@ -205,7 +205,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .window(SlidingEventTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
         def apply(
                    tuple: Tuple,
@@ -230,7 +230,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val window2 = source
       .keyBy(0)
       .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
         def apply(
                    tuple: Tuple,
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
index 60e61f0ae41..daff4a0d368 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
@@ -94,7 +94,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .window(SlidingEventTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of(100): CountTrigger[TimeWindow])
       .reduce(reducer)
 
     val transform1 = window1.javaStream.getTransformation
@@ -109,11 +109,11 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(
       winOperator1.getStateDescriptor.isInstanceOf[ReducingStateDescriptor[_]])
 
-
+val trigger: CountTrigger[TimeWindow] = CountTrigger.of(100);
     val window2 = source
       .keyBy(0)
       .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(trigger)
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
       def apply(
                     tuple: Tuple,
@@ -166,7 +166,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val window2 = source
       .keyBy(0)
       .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .evictor(CountEvictor.of(1000))
       .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
       def apply(
@@ -202,7 +202,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
       .window(SlidingEventTimeWindows.of(
         Time.of(1, TimeUnit.SECONDS),
         Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
         def apply(
                    tuple: Tuple,
@@ -227,7 +227,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
     val window2 = source
       .keyBy(0)
       .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
+      .trigger(CountTrigger.of[TimeWindow](100))
       .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
         def apply(
                    tuple: Tuple,
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
index 9b4855faab8..06c85f40ef9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
@@ -31,8 +31,7 @@
 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.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.triggers.triggerdsl.EventTime;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
@@ -107,7 +106,8 @@ private void runTest(
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		WindowedStream<SessionEvent<Integer, TestEventPayload>, Tuple, TimeWindow> windowedStream
 				= env.addSource(dataSource).keyBy("sessionKey")
-				.window(EventTimeSessionWindows.withGap(Time.milliseconds(MAX_SESSION_EVENT_GAP_MS)));
+				.window(EventTimeSessionWindows.withGap(Time.milliseconds(MAX_SESSION_EVENT_GAP_MS)))
+				.trigger(EventTime.<TimeWindow>Default());
 
 		if (ALLOWED_LATENESS_MS != Long.MAX_VALUE) {
 			windowedStream = windowedStream.allowedLateness(Time.milliseconds(ALLOWED_LATENESS_MS));


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services