You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/03/30 22:04:12 UTC

[01/50] [abbrv] flink git commit: [FLINK-6139] [build] Add "mapr" build profile

Repository: flink
Updated Branches:
  refs/heads/table-retraction d160b5e56 -> ee033c903


[FLINK-6139] [build] Add "mapr" build profile

This build profile is for building Flink to be compatible with MapR. It
additionally excludes dependency clashes between MapR's Hadoop /
Zookeeper distributions and Flink's dependencies.


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

Branch: refs/heads/table-retraction
Commit: a84aa3cd30627e304ea67c38834e1151a335ad09
Parents: aa207ef
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Wed Mar 22 15:14:03 2017 +0800
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Mar 22 22:51:24 2017 +0800

----------------------------------------------------------------------
 .../flink-shaded-hadoop2/pom.xml                | 156 +++++++++++++++++++
 pom.xml                                         |  36 +++++
 2 files changed, 192 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a84aa3cd/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index 86f3f91..c750bbd 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -652,4 +652,160 @@ under the License.
 			</dependency>
 		</dependencies>
 	</dependencyManagement>
+
+	<profiles>
+		<profile>
+			<!-- MapR build profile -->
+			<id>mapr</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+					<version>${hadoop.version}</version>
+					<exclusions>
+						<!--
+							for MapR builds, we additionally exclude the MapR dependencies to not be
+							bundled with Flink, ensuring that the native MapR libraries will be used
+							and avoid incompatibitilies
+						-->
+						<exclusion>
+							<groupId>com.mapr.hadoop</groupId>
+							<artifactId>maprfs-core</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.hadoop</groupId>
+							<artifactId>hadoop2</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.hadoop</groupId>
+							<artifactId>maprfs</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.hadoop</groupId>
+							<artifactId>maprfs-diagnostic-tools</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.hadoop</groupId>
+							<artifactId>maprfs-jni</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.fs</groupId>
+							<artifactId>libprotodefs</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.mapr.fs</groupId>
+							<artifactId>mapr-hbase</artifactId>
+						</exclusion>
+
+						<exclusion>
+							<groupId>asm</groupId>
+							<artifactId>asm</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.ow2.asm</groupId>
+							<artifactId>asm</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>tomcat</groupId>
+							<artifactId>jasper-compiler</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>tomcat</groupId>
+							<artifactId>jasper-runtime</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.mortbay.jetty</groupId>
+							<artifactId>jetty</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.mortbay.jetty</groupId>
+							<artifactId>jsp-api-2.1</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.mortbay.jetty</groupId>
+							<artifactId>jsp-2.1</artifactId>
+						</exclusion>
+
+						<exclusion>
+							<groupId>org.eclipse.jdt</groupId>
+							<artifactId>core</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.mortbay.jetty</groupId>
+							<artifactId>jetty</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey</groupId>
+							<artifactId>jersey-json</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.codehaus.jettison</groupId>
+							<artifactId>jettison</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey</groupId>
+							<artifactId>jersey-server</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>tomcat</groupId>
+							<artifactId>jasper-compiler</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>tomcat</groupId>
+							<artifactId>jasper-runtime</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>javax.servlet.jsp</groupId>
+							<artifactId>jsp-api</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey.jersey-test-framework</groupId>
+							<artifactId>jersey-test-framework-grizzly2</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey.jersey-test-framework</groupId>
+							<artifactId>jersey-test-framework-core</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey</groupId>
+							<artifactId>jersey-grizzly2</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish.grizzly</groupId>
+							<artifactId>grizzly-http</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish.grizzly</groupId>
+							<artifactId>grizzly-framework</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish.grizzly</groupId>
+							<artifactId>grizzly-http-server</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish.grizzly</groupId>
+							<artifactId>grizzly-rcm</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish.grizzly</groupId>
+							<artifactId>grizzly-http-servlet</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>org.glassfish</groupId>
+							<artifactId>javax.servlet</artifactId>
+						</exclusion>
+						<exclusion>
+							<groupId>com.sun.jersey.contribs</groupId>
+							<artifactId>jersey-guice</artifactId>
+						</exclusion>
+						<!--We have to exclude beanutils because it is overlapping with commons-collections-->
+						<exclusion>
+							<groupId>commons-beanutils</groupId>
+							<artifactId>commons-beanutils</artifactId>
+						</exclusion>
+					</exclusions>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/a84aa3cd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5fba16c..17984d3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -522,6 +522,42 @@ under the License.
 		</profile>
 
 		<profile>
+			<!--
+				MapR build profile. This build profile must be used together with "vendor-repos"
+				to be able to locate the MapR Hadoop / Zookeeper dependencies.
+			-->
+			<id>mapr</id>
+
+			<!--
+				use MapR Hadoop / Zookeeper dependencies appropriate for MapR 5.2.0;
+				users of different MapR versions should simply override these versions
+				with appropriate values.
+			-->
+			<properties>
+				<hadoop.version>2.7.0-mapr-1607</hadoop.version>
+				<zookeeper.version>3.4.5-mapr-1604</zookeeper.version>
+			</properties>
+
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.zookeeper</groupId>
+					<artifactId>zookeeper</artifactId>
+					<version>${zookeeper.version}</version>
+					<exclusions>
+						<!--
+							exclude netty, because MapR's Zookeeper distribution has
+							a conflicting Netty version with Flink's Netty dependency
+						-->
+						<exclusion>
+							<groupId>org.jboss.netty</groupId>
+							<artifactId>netty</artifactId>
+						</exclusion>
+					</exclusions>
+				</dependency>
+			</dependencies>
+		</profile>
+
+		<profile>
 			<!-- used for aggregating  ScalaDoc with JavaDoc -->
 			<id>aggregate-scaladoc</id>
 			<dependencies>


[30/50] [abbrv] flink git commit: [hotfix] [docs] Fix broken links

Posted by fh...@apache.org.
[hotfix] [docs] Fix broken links

This closes #3618.


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

Branch: refs/heads/table-retraction
Commit: 037b5cbde18aff9031f1d182f0923e90e0f81f0a
Parents: 662ed33
Author: Patrick Lucas <me...@patricklucas.com>
Authored: Mon Mar 27 11:10:25 2017 +0200
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Mon Mar 27 12:13:16 2017 +0200

----------------------------------------------------------------------
 docs/dev/stream/side_output.md | 2 +-
 docs/dev/windows.md            | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/037b5cbd/docs/dev/stream/side_output.md
----------------------------------------------------------------------
diff --git a/docs/dev/stream/side_output.md b/docs/dev/stream/side_output.md
index 296a4d4..3633b75 100644
--- a/docs/dev/stream/side_output.md
+++ b/docs/dev/stream/side_output.md
@@ -56,7 +56,7 @@ Notice how the `OutputTag` is typed according to the type of elements that the s
 contains.
 
 Emitting data to a side output it only possible when using a
-[ProcessFunction](/dev/stream/process_function.html). In the function, you can use the `Context` parameter
+[ProcessFunction]({{ site.baseurl }}/dev/stream/process_function.html). In the function, you can use the `Context` parameter
 to emit data to a side output identified by an `OutputTag`:
 
 <div class="codetabs" markdown="1">

http://git-wip-us.apache.org/repos/asf/flink/blob/037b5cbd/docs/dev/windows.md
----------------------------------------------------------------------
diff --git a/docs/dev/windows.md b/docs/dev/windows.md
index f8643cd..f0320a1 100644
--- a/docs/dev/windows.md
+++ b/docs/dev/windows.md
@@ -975,7 +975,7 @@ data is ever considered late because the end timestamp of the global window is `
 
 ### Getting late data as a side output
 
-Using Flink's [side output](/dev/stream/side_output.html) feature you can get a stream of the data
+Using Flink's [side output]({{ site.baseurl }}/dev/stream/side_output.html) feature you can get a stream of the data
 that was discarded as late.
 
 You first need to specify that you want to get late data using `sideOutputLateData(OutputTag)` on


[17/50] [abbrv] flink git commit: [FLINK-6182] Fix possible NPE in SourceStreamTask

Posted by fh...@apache.org.
[FLINK-6182] Fix possible NPE in SourceStreamTask

This closes #3606.


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

Branch: refs/heads/table-retraction
Commit: 4b19e272043907b70791bff8a85bd493e212947c
Parents: 11fe3dc
Author: Ufuk Celebi <uc...@apache.org>
Authored: Fri Mar 24 12:24:19 2017 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Fri Mar 24 12:26:54 2017 +0100

----------------------------------------------------------------------
 .../apache/flink/streaming/runtime/tasks/SourceStreamTask.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4b19e272/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
index 63b40ad..1829140 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
@@ -58,6 +58,8 @@ public class SourceStreamTask<OUT, SRC extends SourceFunction<OUT>, OP extends S
 	
 	@Override
 	protected void cancelTask() throws Exception {
-		headOperator.cancel();
+		if (headOperator != null) {
+			headOperator.cancel();
+		}
 	}
 }


[28/50] [abbrv] flink git commit: [FLINK-5929] Allow Access to Per-Window State in ProcessWindowFunction

Posted by fh...@apache.org.
[FLINK-5929] Allow Access to Per-Window State in ProcessWindowFunction


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

Branch: refs/heads/table-retraction
Commit: fad201bfb0b1f2757f68f7b3ffaf97a486eb93e8
Parents: 5c37e55
Author: Seth Wiesman <sw...@mediamath.com>
Authored: Sun Mar 5 23:07:18 2017 -0500
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Sat Mar 25 16:59:17 2017 +0100

----------------------------------------------------------------------
 .../FoldApplyProcessAllWindowFunction.java      |  23 +-
 .../FoldApplyProcessWindowFunction.java         |  23 +-
 .../InternalProcessApplyAllWindowContext.java   |  57 +++++
 .../InternalProcessApplyWindowContext.java      |  58 +++++
 .../windowing/ProcessAllWindowFunction.java     |  22 ++
 .../windowing/ProcessWindowFunction.java        |  24 +-
 .../ReduceApplyProcessAllWindowFunction.java    |  23 +-
 .../ReduceApplyProcessWindowFunction.java       |  21 +-
 .../windowing/AccumulatingKeyedTimePanes.java   |  75 ++++++-
 .../windowing/EvictingWindowOperator.java       |  62 +++---
 .../operators/windowing/WindowOperator.java     | 220 +++++++++++++++----
 ...ternalAggregateProcessAllWindowFunction.java |  28 ++-
 .../InternalAggregateProcessWindowFunction.java |  28 ++-
 .../InternalIterableAllWindowFunction.java      |   7 +-
 ...nternalIterableProcessAllWindowFunction.java |  31 ++-
 .../InternalIterableProcessWindowFunction.java  |  24 +-
 .../InternalIterableWindowFunction.java         |   7 +-
 .../InternalProcessAllWindowContext.java        |  57 +++++
 .../functions/InternalProcessWindowContext.java |  58 +++++
 .../InternalSingleValueAllWindowFunction.java   |   7 +-
 ...rnalSingleValueProcessAllWindowFunction.java |  29 ++-
 ...nternalSingleValueProcessWindowFunction.java |  24 +-
 .../InternalSingleValueWindowFunction.java      |   7 +-
 .../functions/InternalWindowFunction.java       |  26 ++-
 .../FoldApplyProcessWindowFunctionTest.java     |  82 ++++++-
 .../functions/InternalWindowFunctionTest.java   |  49 +++--
 .../RegularWindowOperatorContractTest.java      |  12 +-
 .../windowing/WindowOperatorContractTest.java   | 158 ++++++++++---
 .../function/ProcessAllWindowFunction.scala     |  20 ++
 .../scala/function/ProcessWindowFunction.scala  |  20 ++
 .../ScalaProcessWindowFunctionWrapper.scala     |  31 +++
 31 files changed, 1091 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
index 5ac6766..8e8e52c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
@@ -50,6 +50,7 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
 	private TypeSerializer<ACC> accSerializer;
 	private final TypeInformation<ACC> accTypeInformation;
 	private transient ACC initialValue;
+	private transient InternalProcessApplyAllWindowContext<ACC, R, W> ctx;
 
 	public FoldApplyProcessAllWindowFunction(ACC initialValue, FoldFunction<T, ACC> foldFunction, ProcessAllWindowFunction<ACC, R, W> windowFunction, TypeInformation<ACC> accTypeInformation) {
 		this.windowFunction = windowFunction;
@@ -70,6 +71,9 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
 		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
 		DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais);
 		initialValue = accSerializer.deserialize(in);
+
+		ctx = new InternalProcessApplyAllWindowContext<>(windowFunction);
+
 	}
 
 	@Override
@@ -92,12 +96,19 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
 			result = foldFunction.fold(result, val);
 		}
 
-		windowFunction.process(windowFunction.new Context() {
-			@Override
-			public W window() {
-				return context.window();
-			}
-		}, Collections.singletonList(result), out);
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+
+		windowFunction.process(ctx, Collections.singletonList(result), out);
+	}
+
+	@Override
+	public void clear(final Context context) throws Exception {
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+		windowFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessWindowFunction.java
index e1bc759..073a2f3 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessWindowFunction.java
@@ -50,6 +50,7 @@ public class FoldApplyProcessWindowFunction<K, W extends Window, T, ACC, R>
 	private TypeSerializer<ACC> accSerializer;
 	private final TypeInformation<ACC> accTypeInformation;
 	private transient ACC initialValue;
+	private transient InternalProcessApplyWindowContext<ACC, R, K, W> ctx;
 
 	public FoldApplyProcessWindowFunction(ACC initialValue, FoldFunction<T, ACC> foldFunction, ProcessWindowFunction<ACC, R, K, W> windowFunction, TypeInformation<ACC> accTypeInformation) {
 		this.windowFunction = windowFunction;
@@ -70,6 +71,8 @@ public class FoldApplyProcessWindowFunction<K, W extends Window, T, ACC, R>
 		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
 		DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais);
 		initialValue = accSerializer.deserialize(in);
+
+		ctx = new InternalProcessApplyWindowContext<>(windowFunction);
 	}
 
 	@Override
@@ -85,19 +88,25 @@ public class FoldApplyProcessWindowFunction<K, W extends Window, T, ACC, R>
 	}
 
 	@Override
-	public void process(K key, final Context context, Iterable<T> values, Collector<R> out) throws Exception {
+	public void process(K key, Context context, Iterable<T> values, Collector<R> out) throws Exception {
 		ACC result = accSerializer.copy(initialValue);
 
 		for (T val : values) {
 			result = foldFunction.fold(result, val);
 		}
 
-		windowFunction.process(key, windowFunction.new Context() {
-			@Override
-			public W window() {
-				return context.window();
-			}
-		}, Collections.singletonList(result), out);
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+		windowFunction.process(key, ctx, Collections.singletonList(result), out);
+	}
+
+	@Override
+	public void clear(final Context context) throws Exception{
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+		windowFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java
new file mode 100644
index 0000000..e1a0a98
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.functions.windowing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+/**
+ * Internal reusable context wrapper.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ * @param <W> The type of the window.
+ */
+@Internal
+public class InternalProcessApplyAllWindowContext<IN, OUT, W extends Window>
+	extends ProcessAllWindowFunction<IN, OUT, W>.Context {
+
+	W window;
+	KeyedStateStore windowState;
+	KeyedStateStore globalState;
+
+	InternalProcessApplyAllWindowContext(ProcessAllWindowFunction<IN, OUT, W> function) {
+		function.super();
+	}
+
+	@Override
+	public W window() {
+		return window;
+	}
+
+	@Override
+	public KeyedStateStore windowState() {
+		return windowState;
+	}
+
+	@Override
+	public KeyedStateStore globalState() {
+		return globalState;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyWindowContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyWindowContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyWindowContext.java
new file mode 100644
index 0000000..f547adc
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyWindowContext.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.functions.windowing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+/**
+ * Internal reusable context wrapper.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ * @param <KEY> The type of the window key. 
+ * @param <W> The type of the window.
+ */
+@Internal
+public class InternalProcessApplyWindowContext<IN, OUT, KEY, W extends Window>
+	extends ProcessWindowFunction<IN, OUT, KEY, W>.Context {
+
+	W window;
+	KeyedStateStore windowState;
+	KeyedStateStore globalState;
+
+	InternalProcessApplyWindowContext(ProcessWindowFunction<IN, OUT, KEY, W> function) {
+		function.super();
+	}
+
+	@Override
+	public W window() {
+		return window;
+	}
+
+	@Override
+	public KeyedStateStore windowState() {
+		return windowState;
+	}
+
+	@Override
+	public KeyedStateStore globalState() {
+		return globalState;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
index 622e020..f49aa27 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.functions.windowing;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.state.KeyedStateStore;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
@@ -48,6 +49,14 @@ public abstract class ProcessAllWindowFunction<IN, OUT, W extends Window> implem
 	public abstract void process(Context context, Iterable<IN> elements, Collector<OUT> out) throws Exception;
 
 	/**
+	 * Deletes any state in the {@code Context} when the Window is purged.
+	 *
+	 * @param context The context to which the window is being evaluated
+	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+	 */
+	public void clear(Context context) throws Exception {}
+
+	/**
 	 * The context holding window metadata
 	 */
 	public abstract class Context {
@@ -55,5 +64,18 @@ public abstract class ProcessAllWindowFunction<IN, OUT, W extends Window> implem
 		 * @return The window that is being evaluated.
 		 */
 		public abstract W window();
+
+		/**
+		 * State accessor for per-key and per-window state.
+		 *
+		 * <p><b>NOTE:</b>If you use per-window state you have to ensure that you clean it up
+		 * by implementing {@link ProcessWindowFunction#clear(ProcessWindowFunction.Context)}.
+		 */
+		public abstract KeyedStateStore windowState();
+
+		/**
+		 * State accessor for per-key global state.
+		 */
+		public abstract KeyedStateStore globalState();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessWindowFunction.java
index 9c48e24..bcefaf7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessWindowFunction.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.functions.windowing;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.state.KeyedStateStore;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
@@ -50,12 +51,33 @@ public abstract class ProcessWindowFunction<IN, OUT, KEY, W extends Window> impl
 	public abstract void process(KEY key, Context context, Iterable<IN> elements, Collector<OUT> out) throws Exception;
 
 	/**
+	 * Deletes any state in the {@code Context} when the Window is purged.
+	 *
+	 * @param context The context to which the window is being evaluated
+	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+	 */
+	public void clear(Context context) throws Exception {}
+
+	/**
 	 * The context holding window metadata
 	 */
-	public abstract class Context {
+	public abstract class Context implements java.io.Serializable {
 		/**
 		 * @return The window that is being evaluated.
 		 */
 		public abstract W window();
+
+		/**
+		 * State accessor for per-key and per-window state.
+		 *
+		 * <p><b>NOTE:</b>If you use per-window state you have to ensure that you clean it up
+		 * by implementing {@link ProcessWindowFunction#clear(Context)}.
+		 */
+		public abstract KeyedStateStore windowState();
+
+		/**
+		 * State accessor for per-key global state.
+		 */
+		public abstract KeyedStateStore globalState();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
index 142c71e..4c54c94 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
@@ -35,6 +35,7 @@ public class ReduceApplyProcessAllWindowFunction<W extends Window, T, R>
 
 	private final ReduceFunction<T> reduceFunction;
 	private final ProcessAllWindowFunction<T, R, W> windowFunction;
+	private transient InternalProcessApplyAllWindowContext<T, R, W> ctx;
 
 	public ReduceApplyProcessAllWindowFunction(ReduceFunction<T> reduceFunction, ProcessAllWindowFunction<T, R, W> windowFunction) {
 		this.windowFunction = windowFunction;
@@ -52,17 +53,27 @@ public class ReduceApplyProcessAllWindowFunction<W extends Window, T, R>
 				curr = reduceFunction.reduce(curr, val);
 			}
 		}
-		windowFunction.process(windowFunction.new Context() {
-			@Override
-			public W window() {
-				return context.window();
-			}
-		}, Collections.singletonList(curr), out);
+
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+
+		windowFunction.process(ctx, Collections.singletonList(curr), out);
+	}
+
+	@Override
+	public void clear(final Context context) throws Exception {
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+
+		windowFunction.clear(ctx);
 	}
 
 	@Override
 	public void open(Configuration configuration) throws Exception {
 		FunctionUtils.openFunction(this.windowFunction, configuration);
+		ctx = new InternalProcessApplyAllWindowContext<>(windowFunction);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java
index 9ea1fdf..1af783a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java
@@ -35,6 +35,7 @@ public class ReduceApplyProcessWindowFunction<K, W extends Window, T, R>
 
 	private final ReduceFunction<T> reduceFunction;
 	private final ProcessWindowFunction<T, R, K, W> windowFunction;
+	private transient InternalProcessApplyWindowContext<T, R, K, W> ctx;
 
 	public ReduceApplyProcessWindowFunction(ReduceFunction<T> reduceFunction, ProcessWindowFunction<T, R, K, W> windowFunction) {
 		this.windowFunction = windowFunction;
@@ -52,17 +53,25 @@ public class ReduceApplyProcessWindowFunction<K, W extends Window, T, R>
 				curr = reduceFunction.reduce(curr, val);
 			}
 		}
-		windowFunction.process(k, windowFunction.new Context() {
-			@Override
-			public W window() {
-				return context.window();
-			}
-		}, Collections.singletonList(curr), out);
+
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+		windowFunction.process(k, ctx, Collections.singletonList(curr), out);
+	}
+
+	@Override
+	public void clear(final Context context) throws Exception {
+		this.ctx.window = context.window();
+		this.ctx.windowState = context.windowState();
+		this.ctx.globalState = context.globalState();
+		windowFunction.clear(ctx);
 	}
 
 	@Override
 	public void open(Configuration configuration) throws Exception {
 		FunctionUtils.openFunction(this.windowFunction, configuration);
+		ctx = new InternalProcessApplyWindowContext<>(windowFunction);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
index 87c5aca..d58b5cc 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
@@ -19,6 +19,17 @@
 package org.apache.flink.streaming.runtime.operators.windowing;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
 import org.apache.flink.util.UnionIterator;
@@ -38,6 +49,8 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 
 	private final InternalWindowFunction<Iterable<Type>, Result, Key, Window> function;
 
+	private final AccumulatingKeyedTimePanesContext context;
+
 	/**
 	 * IMPORTANT: This value needs to start at one, so it is fresher than the value that new entries have (zero) */
 	private long evaluationPass = 1L;   
@@ -47,6 +60,7 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 	public AccumulatingKeyedTimePanes(KeySelector<Type, Key> keySelector, InternalWindowFunction<Iterable<Type>, Result, Key, Window> function) {
 		this.keySelector = keySelector;
 		this.function = function;
+		this.context = new AccumulatingKeyedTimePanesContext();
 	}
 
 	// ------------------------------------------------------------------------
@@ -67,13 +81,15 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 			for (KeyMap.Entry<Key, ArrayList<Type>> entry : latestPane) {
 				Key key = entry.getKey();
 				operator.setCurrentKey(key);
-				function.apply(entry.getKey(), window, entry.getValue(), out);
+				context.globalState = operator.getKeyedStateStore();
+
+				function.process(entry.getKey(), window, context, entry.getValue(), out);
 			}
 		}
 		else {
 			// general code path for multi-pane case
 			WindowFunctionTraversal<Key, Type, Result> evaluator = new WindowFunctionTraversal<>(
-					function, window, out, operator);
+					function, window, out, operator, context);
 			traverseAllPanes(evaluator, evaluationPass);
 		}
 		
@@ -95,17 +111,19 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 		private final TimeWindow window;
 		
 		private final AbstractStreamOperator<Result> contextOperator;
-		
+
 		private Key currentKey;
 		
+		private AccumulatingKeyedTimePanesContext context;
 
 		WindowFunctionTraversal(InternalWindowFunction<Iterable<Type>, Result, Key, Window> function, TimeWindow window,
-								Collector<Result> out, AbstractStreamOperator<Result> contextOperator) {
+								Collector<Result> out, AbstractStreamOperator<Result> contextOperator, AccumulatingKeyedTimePanesContext context) {
 			this.function = function;
 			this.out = out;
 			this.unionIterator = new UnionIterator<>();
 			this.window = window;
 			this.contextOperator = contextOperator;
+			this.context = context;
 		}
 
 
@@ -123,7 +141,8 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 		@Override
 		public void keyDone() throws Exception {
 			contextOperator.setCurrentKey(currentKey);
-			function.apply(currentKey, window, unionIterator, out);
+			context.globalState = contextOperator.getKeyedStateStore();
+			function.process(currentKey, window, context, unionIterator, out);
 		}
 	}
 	
@@ -136,6 +155,52 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
 		return (KeyMap.LazyFactory<ArrayList<V>>) LIST_FACTORY;
 	}
 
+	private static class ThrowingKeyedStateStore implements KeyedStateStore {
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not supported when using aligned processing-time windows.");
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not supported when using aligned processing-time windows.");
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not supported when using aligned processing-time windows.");
+		}
+
+		@Override
+		public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T, A> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not supported when using aligned processing-time windows.");
+		}
+
+		@Override
+		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not supported when using aligned processing-time windows.");
+		}
+	}
+
+	private static class AccumulatingKeyedTimePanesContext implements InternalWindowFunction.InternalWindowContext {
+		KeyedStateStore globalState;
+		KeyedStateStore throwingStore;
+
+		public AccumulatingKeyedTimePanesContext() {
+			this.throwingStore = new ThrowingKeyedStateStore();
+		}
+
+		@Override
+		public KeyedStateStore windowState() {
+			return throwingStore;
+		}
+
+		@Override
+		public KeyedStateStore globalState() {
+			return globalState;
+		}
+	}
+
 	private static final KeyMap.LazyFactory<?> LIST_FACTORY = new KeyMap.LazyFactory<ArrayList<?>>() {
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 24c8d32..85451a5 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
@@ -134,14 +134,14 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 											" window: " + mergeResult);
 								}
 
-								context.key = key;
-								context.window = mergeResult;
+								triggerContext.key = key;
+								triggerContext.window = mergeResult;
 
-								context.onMerge(mergedWindows);
+								triggerContext.onMerge(mergedWindows);
 
 								for (W m : mergedWindows) {
-									context.window = m;
-									context.clear();
+									triggerContext.window = m;
+									triggerContext.clear();
 									deleteCleanupTimer(m);
 								}
 
@@ -165,12 +165,12 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				evictingWindowState.setCurrentNamespace(stateWindow);
 				evictingWindowState.add(element);
 
-				context.key = key;
-				context.window = actualWindow;
+				triggerContext.key = key;
+				triggerContext.window = actualWindow;
 				evictorContext.key = key;
 				evictorContext.window = actualWindow;
 
-				TriggerResult triggerResult = context.onElement(element);
+				TriggerResult triggerResult = triggerContext.onElement(element);
 
 				if (triggerResult.isFire()) {
 					Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
@@ -201,12 +201,12 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				evictingWindowState.setCurrentNamespace(window);
 				evictingWindowState.add(element);
 
-				context.key = key;
-				context.window = window;
+				triggerContext.key = key;
+				triggerContext.window = window;
 				evictorContext.key = key;
 				evictorContext.window = window;
 
-				TriggerResult triggerResult = context.onElement(element);
+				TriggerResult triggerResult = triggerContext.onElement(element);
 
 				if (triggerResult.isFire()) {
 					Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
@@ -236,8 +236,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 	@Override
 	public void onEventTime(InternalTimer<K, W> timer) throws Exception {
 
-		context.key = timer.getKey();
-		context.window = timer.getNamespace();
+		triggerContext.key = timer.getKey();
+		triggerContext.window = timer.getNamespace();
 		evictorContext.key = timer.getKey();
 		evictorContext.window = timer.getNamespace();
 
@@ -245,7 +245,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
-			W stateWindow = mergingWindows.getStateWindow(context.window);
+			W stateWindow = mergingWindows.getStateWindow(triggerContext.window);
 			if (stateWindow == null) {
 				// Timer firing for non-existent window, this can only happen if a
 				// trigger did not clean up timers. We have already cleared the merging
@@ -255,23 +255,23 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				evictingWindowState.setCurrentNamespace(stateWindow);
 			}
 		} else {
-			evictingWindowState.setCurrentNamespace(context.window);
+			evictingWindowState.setCurrentNamespace(triggerContext.window);
 		}
 
 		Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 
 		if (contents != null) {
-			TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
+			TriggerResult triggerResult = triggerContext.onEventTime(timer.getTimestamp());
 			if (triggerResult.isFire()) {
-				emitWindowContents(context.window, contents, evictingWindowState);
+				emitWindowContents(triggerContext.window, contents, evictingWindowState);
 			}
 			if (triggerResult.isPurge()) {
 				evictingWindowState.clear();
 			}
 		}
 
-		if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
-			clearAllState(context.window, evictingWindowState, mergingWindows);
+		if (windowAssigner.isEventTime() && isCleanupTime(triggerContext.window, timer.getTimestamp())) {
+			clearAllState(triggerContext.window, evictingWindowState, mergingWindows);
 		}
 
 		if (mergingWindows != null) {
@@ -282,8 +282,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 
 	@Override
 	public void onProcessingTime(InternalTimer<K, W> timer) throws Exception {
-		context.key = timer.getKey();
-		context.window = timer.getNamespace();
+		triggerContext.key = timer.getKey();
+		triggerContext.window = timer.getNamespace();
 		evictorContext.key = timer.getKey();
 		evictorContext.window = timer.getNamespace();
 
@@ -291,7 +291,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
-			W stateWindow = mergingWindows.getStateWindow(context.window);
+			W stateWindow = mergingWindows.getStateWindow(triggerContext.window);
 			if (stateWindow == null) {
 				// Timer firing for non-existent window, this can only happen if a
 				// trigger did not clean up timers. We have already cleared the merging
@@ -301,23 +301,23 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				evictingWindowState.setCurrentNamespace(stateWindow);
 			}
 		} else {
-			evictingWindowState.setCurrentNamespace(context.window);
+			evictingWindowState.setCurrentNamespace(triggerContext.window);
 		}
 
 		Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 
 		if (contents != null) {
-			TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
+			TriggerResult triggerResult = triggerContext.onProcessingTime(timer.getTimestamp());
 			if (triggerResult.isFire()) {
-				emitWindowContents(context.window, contents, evictingWindowState);
+				emitWindowContents(triggerContext.window, contents, evictingWindowState);
 			}
 			if (triggerResult.isPurge()) {
 				evictingWindowState.clear();
 			}
 		}
 
-		if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
-			clearAllState(context.window, evictingWindowState, mergingWindows);
+		if (!windowAssigner.isEventTime() && isCleanupTime(triggerContext.window, timer.getTimestamp())) {
+			clearAllState(triggerContext.window, evictingWindowState, mergingWindows);
 		}
 
 		if (mergingWindows != null) {
@@ -348,7 +348,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				}
 			});
 
-		userFunction.apply(context.key, context.window, projectedContents, timestampedCollector);
+		processContext.window = triggerContext.window;
+		userFunction.process(triggerContext.key, triggerContext.window, processContext, projectedContents, timestampedCollector);
 		evictorContext.evictAfter(recordsWithTimestamp, Iterables.size(recordsWithTimestamp));
 
 
@@ -364,9 +365,10 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 			W window,
 			ListState<StreamRecord<IN>> windowState,
 			MergingWindowSet<W> mergingWindows) throws Exception {
-
 		windowState.clear();
-		context.clear();
+		triggerContext.clear();
+		processContext.window = window;
+		processContext.clear();
 		if (mergingWindows != null) {
 			mergingWindows.retireWindow(window);
 			mergingWindows.persist();

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 3745659..3d40716 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
@@ -23,8 +23,16 @@ import org.apache.commons.math3.util.ArithmeticUtils;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.AppendingState;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.state.MergingState;
+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;
@@ -159,7 +167,9 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	/** This is given to the {@code InternalWindowFunction} for emitting elements with a given timestamp. */
 	protected transient TimestampedCollector<OUT> timestampedCollector;
 
-	protected transient Context context = new Context(null, null);
+	protected transient Context triggerContext = new Context(null, null);
+
+	protected transient WindowContext processContext = new WindowContext(null);
 
 	protected transient WindowAssigner.WindowAssignerContext windowAssignerContext;
 
@@ -264,7 +274,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		internalTimerService =
 				getInternalTimerService("window-timers", windowSerializer, this);
 
-		context = new Context(null, null);
+		triggerContext = new Context(null, null);
+		processContext = new WindowContext( null);
 
 		windowAssignerContext = new WindowAssigner.WindowAssignerContext() {
 			@Override
@@ -317,7 +328,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	public void close() throws Exception {
 		super.close();
 		timestampedCollector = null;
-		context = null;
+		triggerContext = null;
+		processContext = null;
 		windowAssignerContext = null;
 	}
 
@@ -325,7 +337,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	public void dispose() throws Exception {
 		super.dispose();
 		timestampedCollector = null;
-		context = null;
+		triggerContext = null;
+		processContext = null;
 		windowAssignerContext = null;
 	}
 
@@ -365,14 +378,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 									" window: " + mergeResult);
 						}
 
-						context.key = key;
-						context.window = mergeResult;
+						triggerContext.key = key;
+						triggerContext.window = mergeResult;
 
-						context.onMerge(mergedWindows);
+						triggerContext.onMerge(mergedWindows);
 
 						for (W m: mergedWindows) {
-							context.window = m;
-							context.clear();
+							triggerContext.window = m;
+							triggerContext.clear();
 							deleteCleanupTimer(m);
 						}
 
@@ -396,10 +409,10 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				windowState.setCurrentNamespace(stateWindow);
 				windowState.add(element.getValue());
 
-				context.key = key;
-				context.window = actualWindow;
+				triggerContext.key = key;
+				triggerContext.window = actualWindow;
 
-				TriggerResult triggerResult = context.onElement(element);
+				TriggerResult triggerResult = triggerContext.onElement(element);
 
 				if (triggerResult.isFire()) {
 					ACC contents = windowState.get();
@@ -429,10 +442,10 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				windowState.setCurrentNamespace(window);
 				windowState.add(element.getValue());
 
-				context.key = key;
-				context.window = window;
+				triggerContext.key = key;
+				triggerContext.window = window;
 
-				TriggerResult triggerResult = context.onElement(element);
+				TriggerResult triggerResult = triggerContext.onElement(element);
 
 				if (triggerResult.isFire()) {
 					ACC contents = windowState.get();
@@ -460,14 +473,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 
 	@Override
 	public void onEventTime(InternalTimer<K, W> timer) throws Exception {
-		context.key = timer.getKey();
-		context.window = timer.getNamespace();
+		triggerContext.key = timer.getKey();
+		triggerContext.window = timer.getNamespace();
 
 		MergingWindowSet<W> mergingWindows;
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
-			W stateWindow = mergingWindows.getStateWindow(context.window);
+			W stateWindow = mergingWindows.getStateWindow(triggerContext.window);
 			if (stateWindow == null) {
 				// Timer firing for non-existent window, this can only happen if a
 				// trigger did not clean up timers. We have already cleared the merging
@@ -477,7 +490,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				windowState.setCurrentNamespace(stateWindow);
 			}
 		} else {
-			windowState.setCurrentNamespace(context.window);
+			windowState.setCurrentNamespace(triggerContext.window);
 			mergingWindows = null;
 		}
 
@@ -487,17 +500,17 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		}
 
 		if (contents != null) {
-			TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
+			TriggerResult triggerResult = triggerContext.onEventTime(timer.getTimestamp());
 			if (triggerResult.isFire()) {
-				emitWindowContents(context.window, contents);
+				emitWindowContents(triggerContext.window, contents);
 			}
 			if (triggerResult.isPurge()) {
 				windowState.clear();
 			}
 		}
 
-		if (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
-			clearAllState(context.window, windowState, mergingWindows);
+		if (windowAssigner.isEventTime() && isCleanupTime(triggerContext.window, timer.getTimestamp())) {
+			clearAllState(triggerContext.window, windowState, mergingWindows);
 		}
 
 		if (mergingWindows != null) {
@@ -508,14 +521,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 
 	@Override
 	public void onProcessingTime(InternalTimer<K, W> timer) throws Exception {
-		context.key = timer.getKey();
-		context.window = timer.getNamespace();
+		triggerContext.key = timer.getKey();
+		triggerContext.window = timer.getNamespace();
 
 		MergingWindowSet<W> mergingWindows;
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
-			W stateWindow = mergingWindows.getStateWindow(context.window);
+			W stateWindow = mergingWindows.getStateWindow(triggerContext.window);
 			if (stateWindow == null) {
 				// Timer firing for non-existent window, this can only happen if a
 				// trigger did not clean up timers. We have already cleared the merging
@@ -525,7 +538,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				windowState.setCurrentNamespace(stateWindow);
 			}
 		} else {
-			windowState.setCurrentNamespace(context.window);
+			windowState.setCurrentNamespace(triggerContext.window);
 			mergingWindows = null;
 		}
 
@@ -535,17 +548,17 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		}
 
 		if (contents != null) {
-			TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
+			TriggerResult triggerResult = triggerContext.onProcessingTime(timer.getTimestamp());
 			if (triggerResult.isFire()) {
-				emitWindowContents(context.window, contents);
+				emitWindowContents(triggerContext.window, contents);
 			}
 			if (triggerResult.isPurge()) {
 				windowState.clear();
 			}
 		}
 
-		if (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp())) {
-			clearAllState(context.window, windowState, mergingWindows);
+		if (!windowAssigner.isEventTime() && isCleanupTime(triggerContext.window, timer.getTimestamp())) {
+			clearAllState(triggerContext.window, windowState, mergingWindows);
 		}
 
 		if (mergingWindows != null) {
@@ -559,14 +572,16 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 * {@link Trigger#clear(Window, Trigger.TriggerContext)}.
 	 *
 	 * <p>The caller must ensure that the
-	 * correct key is set in the state backend and the context object.
+	 * correct key is set in the state backend and the triggerContext object.
 	 */
 	private void clearAllState(
 			W window,
 			AppendingState<IN, ACC> windowState,
 			MergingWindowSet<W> mergingWindows) throws Exception {
 		windowState.clear();
-		context.clear();
+		triggerContext.clear();
+		processContext.window = window;
+		processContext.clear();
 		if (mergingWindows != null) {
 			mergingWindows.retireWindow(window);
 			mergingWindows.persist();
@@ -579,7 +594,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	@SuppressWarnings("unchecked")
 	private void emitWindowContents(W window, ACC contents) throws Exception {
 		timestampedCollector.setAbsoluteTimestamp(window.maxTimestamp());
-		userFunction.apply(context.key, context.window, contents, timestampedCollector);
+		processContext.window = window;
+		userFunction.process(triggerContext.key, window, processContext, contents, timestampedCollector);
 	}
 
 	/**
@@ -636,9 +652,9 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		}
 
 		if (windowAssigner.isEventTime()) {
-			context.registerEventTimeTimer(cleanupTime);
+			triggerContext.registerEventTimeTimer(cleanupTime);
 		} else {
-			context.registerProcessingTimeTimer(cleanupTime);
+			triggerContext.registerProcessingTimeTimer(cleanupTime);
 		}
 	}
 
@@ -654,9 +670,9 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			return;
 		}
 		if (windowAssigner.isEventTime()) {
-			context.deleteEventTimeTimer(cleanupTime);
+			triggerContext.deleteEventTimeTimer(cleanupTime);
 		} else {
-			context.deleteProcessingTimeTimer(cleanupTime);
+			triggerContext.deleteProcessingTimeTimer(cleanupTime);
 		}
 	}
 
@@ -686,6 +702,134 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	}
 
 	/**
+	 * Base class for per-window {@link KeyedStateStore KeyedStateStores}. Used to allow per-window
+	 * state access for {@link org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction}.
+	 */
+	public abstract class AbstractPerWindowStateStore implements KeyedStateStore {
+
+		// we have this in the base class even though it's not used in MergingKeyStore so that
+		// we can always set it and ignore what actual implementation we have
+		protected W window;
+	}
+
+	/**
+	 * Special {@link AbstractPerWindowStateStore} that doesn't allow access to per-window state.
+	 */
+	public class MergingWindowStateStore extends AbstractPerWindowStateStore {
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not allowed when using merging windows.");
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not allowed when using merging windows.");
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not allowed when using merging windows.");
+		}
+
+		@Override
+		public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T, A> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not allowed when using merging windows.");
+		}
+
+		@Override
+		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
+			throw new UnsupportedOperationException("Per-window state is not allowed when using merging windows.");
+		}
+	}
+
+	/**
+	 * Regular per-window state store for use with
+	 * {@link org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction}.
+	 */
+	public class PerWindowStateStore extends AbstractPerWindowStateStore {
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			try {
+				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state", e);
+			}
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			try {
+				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state", e);
+			}
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			try {
+				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state", e);
+			}
+		}
+
+		@Override
+		public <T, ACC> FoldingState<T, ACC> getFoldingState(FoldingStateDescriptor<T, ACC> stateProperties) {
+			try {
+				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state", e);
+			}
+		}
+
+		@Override
+		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
+			try {
+				return WindowOperator.this.getPartitionedState(window, windowSerializer, stateProperties);
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve state", e);
+			}
+		}
+	}
+
+	/**
+	 * A utility class for handling {@code ProcessWindowFunction} invocations. This can be reused
+	 * by setting the {@code key} and {@code window} fields. No internal state must be kept in the
+	 * {@code WindowContext}.
+	 */
+	public class WindowContext implements InternalWindowFunction.InternalWindowContext {
+		protected W window;
+
+		protected AbstractPerWindowStateStore windowState;
+
+		public WindowContext(W window) {
+			this.window = window;
+			this.windowState = windowAssigner instanceof MergingWindowAssigner ?  new MergingWindowStateStore() : new PerWindowStateStore();
+		}
+
+		@Override
+		public String toString() {
+			return "WindowContext{Window = " + window.toString() + "}";
+		}
+
+		public void clear() throws Exception {
+			userFunction.clear(window, this);
+		}
+
+		@Override
+		public KeyedStateStore windowState() {
+			this.windowState.window = this.window;
+			return this.windowState;
+		}
+
+		@Override
+		public KeyedStateStore globalState() {
+			return WindowOperator.this.getKeyedStateStore();
+		}
+	}
+
+	/**
 	 * {@code Context} is a utility for handling {@code Trigger} invocations. It can be reused
 	 * by setting the {@code key} and {@code window} fields. No internal state must be kept in
 	 * the {@code Context}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java
index 9533c95..83e896d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java
@@ -21,6 +21,7 @@ import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.IterationRuntimeContext;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
@@ -45,6 +46,8 @@ public final class InternalAggregateProcessAllWindowFunction<T, ACC, V, R, W ext
 
 	private final AggregateFunction<T, ACC, V> aggFunction;
 
+	private transient InternalProcessAllWindowContext<V, R, W> ctx;
+
 	public InternalAggregateProcessAllWindowFunction(
 			AggregateFunction<T, ACC, V> aggFunction,
 			ProcessAllWindowFunction<V, R, W> windowFunction) {
@@ -53,22 +56,31 @@ public final class InternalAggregateProcessAllWindowFunction<T, ACC, V, R, W ext
 	}
 
 	@Override
-	public void apply(Byte key, final W window, Iterable<T> input, Collector<R> out) throws Exception {
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
 		ProcessAllWindowFunction<V, R, W> wrappedFunction = this.wrappedFunction;
-		ProcessAllWindowFunction<V, R, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
+		this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction);
+	}
 
+	@Override
+	public void process(Byte key, final W window, final InternalWindowContext context, Iterable<T> input, Collector<R> out) throws Exception {
 		final ACC acc = aggFunction.createAccumulator();
 
 		for (T val : input) {
 			aggFunction.add(val, acc);
 		}
 
-		wrappedFunction.process(context, Collections.singletonList(aggFunction.getResult(acc)), out);
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessAllWindowFunction<V, R, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.process(ctx, Collections.singletonList(aggFunction.getResult(acc)), out);
+	}
+
+	@Override
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java
index 433da9b..e14c9bd 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessWindowFunction.java
@@ -46,30 +46,36 @@ public final class InternalAggregateProcessWindowFunction<T, ACC, V, R, K, W ext
 
 	private final AggregateFunction<T, ACC, V> aggFunction;
 
+	private final InternalProcessWindowContext<V, R, K, W> ctx;
+
 	public InternalAggregateProcessWindowFunction(
 			AggregateFunction<T, ACC, V> aggFunction,
 			ProcessWindowFunction<V, R, K, W> windowFunction) {
 		super(windowFunction);
 		this.aggFunction = aggFunction;
+		this.ctx = new InternalProcessWindowContext<>(windowFunction);
 	}
-	
-	@Override
-	public void apply(K key, final W window, Iterable<T> input, Collector<R> out) throws Exception {
-		ProcessWindowFunction<V, R, K, W> wrappedFunction = this.wrappedFunction;
-		ProcessWindowFunction<V, R, K, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
 
+	@Override
+	public void process(K key, final W window, final InternalWindowContext context, Iterable<T> input, Collector<R> out) throws Exception {
 		final ACC acc = aggFunction.createAccumulator();
 
 		for (T val : input) {
 			aggFunction.add(val, acc);
 		}
 
-		wrappedFunction.process(key, context, Collections.singletonList(aggFunction.getResult(acc)), out);
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessWindowFunction<V, R, K, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.process(key, ctx, Collections.singletonList(aggFunction.getResult(acc)), out);
+	}
+
+	@Override
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessWindowFunction<V, R, K, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java
index 672bdb6..f2507ed 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableAllWindowFunction.java
@@ -39,11 +39,16 @@ public final class InternalIterableAllWindowFunction<IN, OUT, W extends Window>
 	}
 
 	@Override
-	public void apply(Byte key, W window, Iterable<IN> input, Collector<OUT> out) throws Exception {
+	public void process(Byte key, W window, InternalWindowContext context, Iterable<IN> input, Collector<OUT> out) throws Exception {
 		wrappedFunction.apply(window, input, out);
 	}
 
 	@Override
+	public void clear(W window, InternalWindowContext context) throws Exception {
+
+	}
+
+	@Override
 	public RuntimeContext getRuntimeContext() {
 		throw new RuntimeException("This should never be called.");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java
index e33cc2a..47b7d55 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.runtime.operators.windowing.functions;
 import org.apache.flink.api.common.functions.IterationRuntimeContext;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
@@ -34,21 +35,33 @@ public final class InternalIterableProcessAllWindowFunction<IN, OUT, W extends W
 
 	private static final long serialVersionUID = 1L;
 
+	private transient InternalProcessAllWindowContext<IN, OUT, W> ctx;
+
 	public InternalIterableProcessAllWindowFunction(ProcessAllWindowFunction<IN, OUT, W> wrappedFunction) {
 		super(wrappedFunction);
 	}
 
 	@Override
-	public void apply(Byte key, final W window, Iterable<IN> input, Collector<OUT> out) throws Exception {
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
+		this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction);
+	}
+
+	@Override
+	public void process(Byte key, final W window, final InternalWindowContext context, Iterable<IN> input, Collector<OUT> out) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.process(ctx, input, out);
+	}
+
+	@Override
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
 		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
-		ProcessAllWindowFunction<IN, OUT, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
-
-		wrappedFunction.process(context, input, out);
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java
index de516a5..7eb015e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessWindowFunction.java
@@ -34,21 +34,27 @@ public final class InternalIterableProcessWindowFunction<IN, OUT, KEY, W extends
 
 	private static final long serialVersionUID = 1L;
 
+	private final InternalProcessWindowContext<IN, OUT, KEY, W> ctx;
+
 	public InternalIterableProcessWindowFunction(ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction) {
 		super(wrappedFunction);
+		this.ctx = new InternalProcessWindowContext<>(wrappedFunction);
+	}
+
+	@Override
+	public void process(KEY key, final W window, final InternalWindowContext context, Iterable<IN> input, Collector<OUT> out) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.process(key, ctx, input, out);
 	}
 
 	@Override
-	public void apply(KEY key, final W window, Iterable<IN> input, Collector<OUT> out) throws Exception {
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
 		ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction = this.wrappedFunction;
-		ProcessWindowFunction<IN, OUT, KEY, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
-		
-		wrappedFunction.process(key, context, input, out);
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java
index 895b31f..e2f1517 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableWindowFunction.java
@@ -39,11 +39,16 @@ public final class InternalIterableWindowFunction<IN, OUT, KEY, W extends Window
 	}
 
 	@Override
-	public void apply(KEY key, W window, Iterable<IN> input, Collector<OUT> out) throws Exception {
+	public void process(KEY key, W window, InternalWindowContext context, Iterable<IN> input, Collector<OUT> out) throws Exception {
 		wrappedFunction.apply(key, window, input, out);
 	}
 
 	@Override
+	public void clear(W window, InternalWindowContext context) throws Exception {
+
+	}
+
+	@Override
 	public RuntimeContext getRuntimeContext() {
 		throw new RuntimeException("This should never be called.");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java
new file mode 100644
index 0000000..c70e161
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java
@@ -0,0 +1,57 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+/**
+ * Internal reusable context wrapper.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ * @param <W> The type of the window.
+ */
+@Internal
+public class InternalProcessAllWindowContext<IN, OUT, W extends Window>
+	extends ProcessAllWindowFunction<IN, OUT, W>.Context {
+
+	W window;
+	InternalWindowFunction.InternalWindowContext internalContext;
+
+	InternalProcessAllWindowContext(ProcessAllWindowFunction<IN, OUT, W> function) {
+		function.super();
+	}
+
+	@Override
+	public W window() {
+		return window;
+	}
+
+	@Override
+	public KeyedStateStore windowState() {
+		return internalContext.windowState();
+	}
+
+	@Override
+	public KeyedStateStore globalState() {
+		return internalContext.globalState();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessWindowContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessWindowContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessWindowContext.java
new file mode 100644
index 0000000..0f1c0ee
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessWindowContext.java
@@ -0,0 +1,58 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+/**
+ * Internal reusable context wrapper.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ * @param <KEY> The type of the key.
+ * @param <W> The type of the window.
+ */
+@Internal
+public class InternalProcessWindowContext<IN, OUT, KEY, W extends Window>
+	extends ProcessWindowFunction<IN, OUT, KEY, W>.Context {
+
+	W window;
+	InternalWindowFunction.InternalWindowContext internalContext;
+
+	InternalProcessWindowContext(ProcessWindowFunction<IN, OUT, KEY, W> function) {
+		function.super();
+	}
+
+	@Override
+	public W window() {
+		return window;
+	}
+
+	@Override
+	public KeyedStateStore windowState() {
+		return internalContext.windowState();
+	}
+
+	@Override
+	public KeyedStateStore globalState() {
+		return internalContext.globalState();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java
index a34d3ec..e90bcf4 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueAllWindowFunction.java
@@ -41,11 +41,16 @@ public final class InternalSingleValueAllWindowFunction<IN, OUT, W extends Windo
 	}
 
 	@Override
-	public void apply(Byte key, W window, IN input, Collector<OUT> out) throws Exception {
+	public void process(Byte key, W window, InternalWindowContext context, IN input, Collector<OUT> out) throws Exception {
 		wrappedFunction.apply(window, Collections.singletonList(input), out);
 	}
 
 	@Override
+	public void clear(W window, InternalWindowContext context) throws Exception {
+
+	}
+
+	@Override
 	public RuntimeContext getRuntimeContext() {
 		throw new RuntimeException("This should never be called.");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java
index 0284ef7..f7c6a08 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.runtime.operators.windowing.functions;
 import org.apache.flink.api.common.functions.IterationRuntimeContext;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
@@ -36,21 +37,33 @@ public final class InternalSingleValueProcessAllWindowFunction<IN, OUT, W extend
 
 	private static final long serialVersionUID = 1L;
 
+	private transient InternalProcessAllWindowContext<IN, OUT, W> ctx;
+
 	public InternalSingleValueProcessAllWindowFunction(ProcessAllWindowFunction<IN, OUT, W> wrappedFunction) {
 		super(wrappedFunction);
 	}
 
 	@Override
-	public void apply(Byte key, final W window, IN input, Collector<OUT> out) throws Exception {
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
+		this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction);
+	}
+
+	@Override
+	public void process(Byte key, final W window, final InternalWindowContext context, IN input, Collector<OUT> out) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
 		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
-		ProcessAllWindowFunction<IN, OUT, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
+		wrappedFunction.process(ctx, Collections.singletonList(input), out);
+	}
 
-		wrappedFunction.process(context, Collections.singletonList(input), out);
+	@Override
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+		ProcessAllWindowFunction<IN, OUT, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java
index 7a4e8c6..21d1639 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessWindowFunction.java
@@ -36,21 +36,29 @@ public final class InternalSingleValueProcessWindowFunction<IN, OUT, KEY, W exte
 
 	private static final long serialVersionUID = 1L;
 
+	private final InternalProcessWindowContext<IN, OUT, KEY, W> ctx;
+
 	public InternalSingleValueProcessWindowFunction(ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction) {
 		super(wrappedFunction);
+		ctx = new InternalProcessWindowContext<>(wrappedFunction);
 	}
 
 	@Override
-	public void apply(KEY key, final W window, IN input, Collector<OUT> out) throws Exception {
+	public void process(KEY key, final W window, final InternalWindowContext context, IN input, Collector<OUT> out) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+
 		ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction = this.wrappedFunction;
-		ProcessWindowFunction<IN, OUT, KEY, W>.Context context = wrappedFunction.new Context() {
-			@Override
-			public W window() {
-				return window;
-			}
-		};
+		wrappedFunction.process(key, ctx, Collections.singletonList(input), out);
+	}
 
-		wrappedFunction.process(key, context, Collections.singletonList(input), out);
+	@Override
+	public void clear(final W window, final InternalWindowContext context) throws Exception {
+		this.ctx.window = window;
+		this.ctx.internalContext = context;
+
+		ProcessWindowFunction<IN, OUT, KEY, W> wrappedFunction = this.wrappedFunction;
+		wrappedFunction.clear(ctx);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java
index 9a0a447..d5cc4a2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueWindowFunction.java
@@ -41,11 +41,16 @@ public final class InternalSingleValueWindowFunction<IN, OUT, KEY, W extends Win
 	}
 
 	@Override
-	public void apply(KEY key, W window, IN input, Collector<OUT> out) throws Exception {
+	public void process(KEY key, W window, InternalWindowContext context, IN input, Collector<OUT> out) throws Exception {
 		wrappedFunction.apply(key, window, Collections.singletonList(input), out);
 	}
 
 	@Override
+	public void clear(W window, InternalWindowContext context) throws Exception {
+
+	}
+
+	@Override
 	public RuntimeContext getRuntimeContext() {
 		throw new RuntimeException("This should never be called.");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalWindowFunction.java
index 2eb4052..9834480 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalWindowFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalWindowFunction.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.runtime.operators.windowing.functions;
 
 import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.state.KeyedStateStore;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
@@ -29,15 +30,28 @@ import org.apache.flink.util.Collector;
  * @param <KEY> The type of the key.
  */
 public interface InternalWindowFunction<IN, OUT, KEY, W extends Window> extends Function {
-
 	/**
 	 * Evaluates the window and outputs none or several elements.
 	 *
-	 * @param key    The key for which this window is evaluated.
-	 * @param window The window that is being evaluated.
-	 * @param input  The elements in the window being evaluated.
-	 * @param out    A collector for emitting elements.
+	 * @param context The context in which the window is being evaluated.
+	 * @param input The elements in the window being evaluated.
+	 * @param out A collector for emitting elements.
+	 *
 	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
 	 */
-	void apply(KEY key, W window, IN input, Collector<OUT> out) throws Exception;
+	void process(KEY key, W window, InternalWindowContext context, IN input, Collector<OUT> out) throws Exception;
+
+	/**
+	 * Deletes any state in the {@code Context} when the Window is purged.
+	 *
+	 * @param context The context to which the window is being evaluated
+	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+	 */
+	void clear(W window, InternalWindowContext context) throws Exception;
+
+	interface InternalWindowContext extends java.io.Serializable {
+		KeyedStateStore windowState();
+
+		KeyedStateStore globalState();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
index 4b479f3..c4bed37 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
@@ -21,20 +21,28 @@ package org.apache.flink.streaming.api.operators;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.ByteSerializer;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.FoldApplyProcessAllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.FoldApplyProcessWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
@@ -45,8 +53,8 @@ import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProces
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableProcessAllWindowFunction;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableProcessWindowFunction;
 import org.apache.flink.util.Collector;
-import org.junit.Test;
 import org.junit.Assert;
+import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -139,12 +147,26 @@ public class FoldApplyProcessWindowFunctionTest {
 
 		expected.add(initValue);
 
-		foldWindowFunction.process(0, foldWindowFunction.new Context() {
+		FoldApplyProcessWindowFunction<Integer, TimeWindow, Integer, Integer, Integer>.Context ctx = foldWindowFunction.new Context() {
 			@Override
 			public TimeWindow window() {
 				return new TimeWindow(0, 1);
 			}
-		}, input, new ListCollector<>(result));
+
+			@Override
+			public KeyedStateStore windowState() {
+				return new DummyKeyedStateStore();
+			}
+
+			@Override
+			public KeyedStateStore globalState() {
+				return new DummyKeyedStateStore();
+			}
+		};
+
+		foldWindowFunction.open(new Configuration());
+
+		foldWindowFunction.process(0, ctx, input, new ListCollector<>(result));
 
 		Assert.assertEquals(expected, result);
 	}
@@ -234,16 +256,58 @@ public class FoldApplyProcessWindowFunctionTest {
 
 		expected.add(initValue);
 
-		foldWindowFunction.process(foldWindowFunction.new Context() {
+		FoldApplyProcessAllWindowFunction<TimeWindow, Integer, Integer, Integer>.Context ctx = foldWindowFunction.new Context() {
 			@Override
 			public TimeWindow window() {
 				return new TimeWindow(0, 1);
 			}
-		}, input, new ListCollector<>(result));
+
+			@Override
+			public KeyedStateStore windowState() {
+				return new DummyKeyedStateStore();
+			}
+
+			@Override
+			public KeyedStateStore globalState() {
+				return new DummyKeyedStateStore();
+			}
+		};
+
+		foldWindowFunction.open(new Configuration());
+
+		foldWindowFunction.process(ctx, input, new ListCollector<>(result));
 
 		Assert.assertEquals(expected, result);
 	}
 
+	public static class DummyKeyedStateStore implements KeyedStateStore {
+
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			return null;
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			return null;
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			return null;
+		}
+
+		@Override
+		public <T, ACC> FoldingState<T, ACC> getFoldingState(FoldingStateDescriptor<T, ACC> stateProperties) {
+			return null;
+		}
+
+		@Override
+		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
+			return null;
+		}
+	}
+
 	public static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
 
 		public DummyStreamExecutionEnvironment() {


[40/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Clean up exception signature of ExecutionGraph

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Clean up exception signature of ExecutionGraph


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

Branch: refs/heads/table-retraction
Commit: 231bec8d4d5b2e79e279ea1acfd9d4f14f892728
Parents: c277ee1
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 29 17:09:48 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 17:11:49 2017 +0200

----------------------------------------------------------------------
 .../flink/runtime/executiongraph/ExecutionGraph.java  | 14 +++++++++++---
 .../runtime/executiongraph/ExecutionGraphBuilder.java | 10 ++--------
 2 files changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/231bec8d/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 06b2f9a..b21b72b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -63,6 +63,7 @@ import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.SerializedValue;
 
@@ -240,7 +241,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			SerializedValue<ExecutionConfig> serializedConfig,
 			Time timeout,
 			RestartStrategy restartStrategy,
-			SlotProvider slotProvider) throws IOException {
+			SlotProvider slotProvider) {
 		this(
 			futureExecutor,
 			ioExecutor,
@@ -268,7 +269,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			List<BlobKey> requiredJarFiles,
 			List<URL> requiredClasspaths,
 			SlotProvider slotProvider,
-			ClassLoader userClassLoader) throws IOException {
+			ClassLoader userClassLoader) {
 
 		checkNotNull(futureExecutor);
 		checkNotNull(jobId);
@@ -284,7 +285,14 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			requiredClasspaths);
 
 		// serialize the job information to do the serialisation work only once
-		this.serializedJobInformation = new SerializedValue<>(jobInformation);
+		try {
+			this.serializedJobInformation = new SerializedValue<>(jobInformation);
+		}
+		catch (IOException e) {
+			// this cannot happen because 'JobInformation' is perfectly serializable
+			// rethrow unchecked, because this indicates a bug, not a recoverable situation
+			throw new FlinkRuntimeException("Bug: Cannot serialize JobInformation", e);
+		}
 
 		this.futureExecutor = Preconditions.checkNotNull(futureExecutor);
 		this.ioExecutor = Preconditions.checkNotNull(ioExecutor);

http://git-wip-us.apache.org/repos/asf/flink/blob/231bec8d/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
index 494b7a2..f1da8bd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -87,11 +87,8 @@ public class ExecutionGraphBuilder {
 		final JobID jobId = jobGraph.getJobID();
 
 		// create a new execution graph, if none exists so far
-		final ExecutionGraph executionGraph;
-
-		try {
-			executionGraph = (prior != null) ? prior :
-					new ExecutionGraph(
+		final ExecutionGraph executionGraph = (prior != null) ? prior :
+				new ExecutionGraph(
 						futureExecutor,
 						ioExecutor,
 						jobId,
@@ -104,9 +101,6 @@ public class ExecutionGraphBuilder {
 						jobGraph.getClasspaths(),
 						slotProvider,
 						classLoader);
-		} catch (IOException e) {
-			throw new JobException("Could not create the execution graph.", e);
-		}
 
 		// set the basic properties
 


[22/50] [abbrv] flink git commit: [hotfix] [table] Improved code documentation for external catalog.

Posted by fh...@apache.org.
[hotfix] [table] Improved code documentation for external catalog.


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

Branch: refs/heads/table-retraction
Commit: f97deaa9683bf1868ecf104c73b997ede63e8856
Parents: 135a57c
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu Mar 23 22:18:37 2017 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 20:19:17 2017 +0100

----------------------------------------------------------------------
 .../flink/table/annotation/TableType.java       |  6 +-
 .../table/catalog/CrudExternalCatalog.scala     | 86 ++++++++++----------
 .../flink/table/catalog/ExternalCatalog.scala   | 40 ++++-----
 .../table/catalog/ExternalCatalogDatabase.scala |  6 +-
 .../table/catalog/ExternalCatalogTable.scala    | 24 +++---
 .../table/catalog/TableSourceConverter.scala    | 23 +++---
 6 files changed, 94 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/java/org/apache/flink/table/annotation/TableType.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/annotation/TableType.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/annotation/TableType.java
index 1cebe53..3845eae 100644
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/annotation/TableType.java
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/annotation/TableType.java
@@ -27,7 +27,7 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
 /**
- * A {@link TableSourceConverter} with this annotation bind the converter with table type.
+ * Annotates a table type of a {@link TableSourceConverter}.
  */
 @Documented
 @Target(ElementType.TYPE)
@@ -36,9 +36,9 @@ import java.lang.annotation.Target;
 public @interface TableType {
 
 	/**
-	 * Specifies the external catalog table type of {@link TableSourceConverter}.
+	 * Returns the table type of a {@link TableSourceConverter}.
 	 *
-	 * @return the external catalog table type of {@link TableSourceConverter}.
+	 * @return The table type of the {@link TableSourceConverter}.
 	 */
 	String value();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CrudExternalCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CrudExternalCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CrudExternalCatalog.scala
index d93f140..fcefa45 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CrudExternalCatalog.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CrudExternalCatalog.scala
@@ -21,88 +21,86 @@ package org.apache.flink.table.catalog
 import org.apache.flink.table.api._
 
 /**
-  * This class is responsible for interact with external catalog.
-  * Its main responsibilities including:
-  * <ul>
-  * <li> create/drop/alter database or tables for DDL operations
-  * <li> provide tables for calcite catalog, it looks up databases or tables in the external catalog
-  * </ul>
+  * The CrudExternalCatalog provides methods to create, drop, and alter databases or tables.
   */
 trait CrudExternalCatalog extends ExternalCatalog {
 
   /**
-    * Adds table into external Catalog
+    * Adds a table to the catalog.
     *
-    * @param table          description of table which to create
-    * @param ignoreIfExists if table already exists in the catalog, not throw exception and leave
-    *                       the existed table if ignoreIfExists is true;
-    *                       else throw a TableAlreadyExistException.
-    * @throws DatabaseNotExistException  if database does not exist in the catalog yet
-    * @throws TableAlreadyExistException if table already exists in the catalog and
-    *                                    ignoreIfExists is false
+    * @param table          Description of the table to add
+    * @param ignoreIfExists Flag to specify behavior if a table with the given name already exists:
+    *                       if set to false, it throws a TableAlreadyExistException,
+    *                       if set to true, nothing happens.
+    * @throws DatabaseNotExistException  thrown if database does not exist
+    * @throws TableAlreadyExistException thrown if table already exists and ignoreIfExists is false
     */
   @throws[DatabaseNotExistException]
   @throws[TableAlreadyExistException]
   def createTable(table: ExternalCatalogTable, ignoreIfExists: Boolean): Unit
 
   /**
-    * Deletes table from external Catalog
+    * Deletes table from a database of the catalog.
     *
-    * @param dbName            database name
-    * @param tableName         table name
-    * @param ignoreIfNotExists if table not exist yet, not throw exception if ignoreIfNotExists is
-    *                          true; else throw TableNotExistException
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
-    * @throws TableNotExistException    if table does not exist in the catalog yet
+    * @param dbName            Name of the database
+    * @param tableName         Name of the table
+    * @param ignoreIfNotExists Flag to specify behavior if the table or database does not exist:
+    *                          if set to false, throw an exception,
+    *                          if set to true, nothing happens.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
+    * @throws TableNotExistException    thrown if the table does not exist in the catalog
     */
   @throws[DatabaseNotExistException]
   @throws[TableNotExistException]
   def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean): Unit
 
   /**
-    * Modifies an existing table in the external catalog
+    * Modifies an existing table in the catalog.
     *
-    * @param table             description of table which to modify
-    * @param ignoreIfNotExists if the table not exist yet, not throw exception if ignoreIfNotExists
-    *                          is true; else throw TableNotExistException
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
-    * @throws TableNotExistException    if table does not exist in the catalog yet
+    * @param table             New description of the table to update
+    * @param ignoreIfNotExists Flag to specify behavior if the table or database does not exist:
+    *                          if set to false, throw an exception,
+    *                          if set to true, nothing happens.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
+    * @throws TableNotExistException    thrown if the table does not exist in the catalog
     */
   @throws[DatabaseNotExistException]
   @throws[TableNotExistException]
   def alterTable(table: ExternalCatalogTable, ignoreIfNotExists: Boolean): Unit
 
   /**
-    * Adds database into external Catalog
+    * Adds a database to the catalog.
     *
-    * @param db             description of database which to create
-    * @param ignoreIfExists if database already exists in the catalog, not throw exception and leave
-    *                       the existed database if ignoreIfExists is true;
-    *                       else throw a DatabaseAlreadyExistException.
-    * @throws DatabaseAlreadyExistException if database already exists in the catalog and
-    *                                       ignoreIfExists is false
+    * @param db             Description of the database to create
+    * @param ignoreIfExists Flag to specify behavior if a database with the given name already
+    *                       exists: if set to false, it throws a DatabaseAlreadyExistException,
+    *                       if set to true, nothing happens.
+    * @throws DatabaseAlreadyExistException thrown if the database does already exist in the catalog
+    *                                       and ignoreIfExists is false
     */
   @throws[DatabaseAlreadyExistException]
   def createDatabase(db: ExternalCatalogDatabase, ignoreIfExists: Boolean): Unit
 
   /**
-    * Deletes database from external Catalog
+    * Deletes a database from the catalog.
     *
-    * @param dbName            database name
-    * @param ignoreIfNotExists if database not exist yet, not throw exception if ignoreIfNotExists
-    *                          is true; else throw DatabaseNotExistException
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
+    * @param dbName            Name of the database.
+    * @param ignoreIfNotExists Flag to specify behavior if the database does not exist:
+    *                          if set to false, throw an exception,
+    *                          if set to true, nothing happens.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
     */
   @throws[DatabaseNotExistException]
   def dropDatabase(dbName: String, ignoreIfNotExists: Boolean): Unit
 
   /**
-    * Modifies existed database into external Catalog
+    * Modifies an existing database in the catalog.
     *
-    * @param db                description of database which to modify
-    * @param ignoreIfNotExists if database not exist yet, not throw exception if ignoreIfNotExists
-    *                          is true; else throw DatabaseNotExistException
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
+    * @param db                New description of the database to update
+    * @param ignoreIfNotExists Flag to specify behavior if the database does not exist:
+    *                          if set to false, throw an exception,
+    *                          if set to true, nothing happens.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
     */
   @throws[DatabaseNotExistException]
   def alterDatabase(db: ExternalCatalogDatabase, ignoreIfNotExists: Boolean): Unit

http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala
index 58b62c7..00a35e4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala
@@ -23,49 +23,51 @@ import java.util.{List => JList}
 import org.apache.flink.table.api._
 
 /**
-  * This class is responsible for read table/database from external catalog.
-  * Its main responsibilities is provide tables for calcite catalog, it looks up databases or tables
-  * in the external catalog.
+  * An [[ExternalCatalog]] is the connector between an external database catalog and Flink's
+  * Table API.
+  *
+  * It provides information about databases and tables such as names, schema, statistics, and
+  * access information.
   */
 trait ExternalCatalog {
 
   /**
-    * Gets table from external Catalog
+    * Get a table from the catalog
     *
-    * @param dbName    database name
-    * @param tableName table name
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
-    * @throws TableNotExistException    if table does not exist in the catalog yet
-    * @return found table
+    * @param dbName    The name of the table's database.
+    * @param tableName The name of the table.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog.
+    * @throws TableNotExistException    thrown if the table does not exist in the catalog.
+    * @return the requested table
     */
   @throws[DatabaseNotExistException]
   @throws[TableNotExistException]
   def getTable(dbName: String, tableName: String): ExternalCatalogTable
 
   /**
-    * Gets the table name lists from current external Catalog
+    * Get a list of all table names of a database in the catalog.
     *
-    * @param dbName database name
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
-    * @return lists of table name
+    * @param dbName The name of the database.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
+    * @return The list of table names
     */
   @throws[DatabaseNotExistException]
   def listTables(dbName: String): JList[String]
 
   /**
-    * Gets database from external Catalog
+    * Gets a database from the catalog.
     *
-    * @param dbName database name
-    * @throws DatabaseNotExistException if database does not exist in the catalog yet
-    * @return found database
+    * @param dbName The name of the database.
+    * @throws DatabaseNotExistException thrown if the database does not exist in the catalog
+    * @return The requested database
     */
   @throws[DatabaseNotExistException]
   def getDatabase(dbName: String): ExternalCatalogDatabase
 
   /**
-    * Gets the database name lists from current external Catalog
+    * Gets a list of all databases in the catalog.
     *
-    * @return list of database names
+    * @return The list of database names
     */
   def listDatabases(): JList[String]
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogDatabase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogDatabase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogDatabase.scala
index c2a4702..99ab2eb 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogDatabase.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogDatabase.scala
@@ -21,10 +21,10 @@ package org.apache.flink.table.catalog
 import java.util.{HashMap => JHashMap, Map => JMap}
 
 /**
-  * Database definition of the external catalog.
+  * Defines a database in an [[ExternalCatalog]].
   *
-  * @param dbName     database name
-  * @param properties database properties
+  * @param dbName     The name of the database
+  * @param properties The properties of the database
   */
 case class ExternalCatalogDatabase(
     dbName: String,

http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala
index 893cbb3..4fdab66 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala
@@ -26,16 +26,16 @@ import org.apache.flink.table.api.TableSchema
 import org.apache.flink.table.plan.stats.TableStats
 
 /**
-  * Table definition of the external catalog.
+  * Defines a table in an [[ExternalCatalog]].
   *
-  * @param identifier           identifier of external catalog table, including dbName and tableName
-  * @param tableType            type of external catalog table, e.g csv, hbase, kafka
-  * @param schema               schema of table data, including column names and column types
-  * @param properties           properties of external catalog table
-  * @param stats                statistics of external catalog table
-  * @param comment              comment of external catalog table
-  * @param createTime           create time of external catalog table
-  * @param lastAccessTime       last access time of of external catalog table
+  * @param identifier           Identifier of the table (database name and table name)
+  * @param tableType            Table type, e.g csv, hbase, kafka
+  * @param schema               Schema of the table (column names and types)
+  * @param properties           Properties of the table
+  * @param stats                Statistics of the table
+  * @param comment              Comment of the table
+  * @param createTime           Create timestamp of the table
+  * @param lastAccessTime       Timestamp of last access of the table
   */
 case class ExternalCatalogTable(
     identifier: TableIdentifier,
@@ -48,10 +48,10 @@ case class ExternalCatalogTable(
     lastAccessTime: JLong = -1L)
 
 /**
-  * Identifier of external catalog table
+  * Identifier for a catalog table.
   *
-  * @param database database name
-  * @param table    table name
+  * @param database Database name
+  * @param table    Table name
   */
 case class TableIdentifier(
     database: String,

http://git-wip-us.apache.org/repos/asf/flink/blob/f97deaa9/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/TableSourceConverter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/TableSourceConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/TableSourceConverter.scala
index 13e54a6..ca6df9a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/TableSourceConverter.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/TableSourceConverter.scala
@@ -22,26 +22,29 @@ import java.util.{Set => JSet}
 
 import org.apache.flink.table.sources.TableSource
 
-/** Defines a converter used to convert [[org.apache.flink.table.sources.TableSource]] to
-  * or from [[ExternalCatalogTable]].
+/** Creates a [[org.apache.flink.table.sources.TableSource]] from the properties of an
+  * [[ExternalCatalogTable]].
   *
-  * @tparam T The tableSource which to do convert operation on.
+  * The [[org.apache.flink.table.annotation.TableType]] annotation defines which type of external
+  * table is supported.
+  *
+  * @tparam T The [[TableSource]] to be created by this converter.
   */
 trait TableSourceConverter[T <: TableSource[_]] {
 
   /**
-    * Defines the required properties that must exists in the properties of an ExternalCatalogTable
-    * to ensure the input ExternalCatalogTable is compatible with the requirements of
-    * current converter.
-    * @return the required properties.
+    * Defines the properties that need to be provided by the [[ExternalCatalogTable]] to create
+    * the [[TableSource]].
+    *
+    * @return The required properties.
     */
   def requiredProperties: JSet[String]
 
   /**
-    * Converts the input external catalog table instance to a tableSource instance.
+    * Creates a [[TableSource]] for the given [[ExternalCatalogTable]].
     *
-    * @param externalCatalogTable input external catalog table instance to convert
-    * @return converted tableSource instance from input external catalog table.
+    * @param externalCatalogTable ExternalCatalogTable to create a TableSource from.
+    * @return The created TableSource.
     */
   def fromExternalCatalogTable(externalCatalogTable: ExternalCatalogTable): T
 


[21/50] [abbrv] flink git commit: [FLINK-5990] [table] Add event-time OVER ROWS BETWEEN x PRECEDING aggregation to SQL.

Posted by fh...@apache.org.
[FLINK-5990] [table] Add event-time OVER ROWS BETWEEN x PRECEDING aggregation to SQL.

This closes #3585.


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

Branch: refs/heads/table-retraction
Commit: 7a9d39fe9f659d43bf4719a2981f6c4771ffbe48
Parents: 6949c8c
Author: \u91d1\u7af9 <ji...@alibaba-inc.com>
Authored: Sun Mar 19 23:31:00 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 20:19:17 2017 +0100

----------------------------------------------------------------------
 .../flink/table/plan/nodes/OverAggregate.scala  |  31 ++-
 .../datastream/DataStreamOverAggregate.scala    | 149 +++++++++---
 .../table/runtime/aggregate/AggregateUtil.scala |  48 +++-
 .../RowsClauseBoundedOverProcessFunction.scala  | 239 +++++++++++++++++++
 .../table/api/scala/stream/sql/SqlITCase.scala  | 139 ++++++++++-
 .../scala/stream/sql/WindowAggregateTest.scala  |  55 +++++
 6 files changed, 623 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
index 793ab23..91c8cef 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/OverAggregate.scala
@@ -18,12 +18,15 @@
 
 package org.apache.flink.table.plan.nodes
 
-import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.{RelFieldCollation, RelNode}
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl}
 import org.apache.calcite.rel.core.AggregateCall
 import org.apache.calcite.rel.core.Window.Group
+import org.apache.calcite.rel.core.Window
+import org.apache.calcite.rex.{RexInputRef}
 import org.apache.flink.table.runtime.aggregate.AggregateUtil._
 import org.apache.flink.table.functions.{ProcTimeType, RowTimeType}
+
 import scala.collection.JavaConverters._
 
 trait OverAggregate {
@@ -46,8 +49,16 @@ trait OverAggregate {
     orderingString
   }
 
-  private[flink] def windowRange(overWindow: Group): String = {
-    s"BETWEEN ${overWindow.lowerBound} AND ${overWindow.upperBound}"
+  private[flink] def windowRange(
+    logicWindow: Window,
+    overWindow: Group,
+    input: RelNode): String = {
+    if (overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded) {
+      s"BETWEEN ${getLowerBoundary(logicWindow, overWindow, input)} PRECEDING " +
+          s"AND ${overWindow.upperBound}"
+    } else {
+      s"BETWEEN ${overWindow.lowerBound} AND ${overWindow.upperBound}"
+    }
   }
 
   private[flink] def aggregationToString(
@@ -92,4 +103,18 @@ trait OverAggregate {
     }.mkString(", ")
   }
 
+  private[flink] def getLowerBoundary(
+    logicWindow: Window,
+    overWindow: Group,
+    input: RelNode): Long = {
+
+    val ref: RexInputRef = overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef]
+    val lowerBoundIndex = input.getRowType.getFieldCount - ref.getIndex;
+    val lowerBound = logicWindow.constants.get(lowerBoundIndex).getValue2
+    lowerBound match {
+      case x: java.math.BigDecimal => x.asInstanceOf[java.math.BigDecimal].longValue()
+      case _ => lowerBound.asInstanceOf[Long]
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 34b3b0f..547c875 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -32,6 +32,7 @@ import org.apache.calcite.rel.core.Window
 import org.apache.calcite.rel.core.Window.Group
 import java.util.{List => JList}
 
+import org.apache.flink.api.java.functions.NullByteKeySelector
 import org.apache.flink.table.functions.{ProcTimeType, RowTimeType}
 import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
 
@@ -70,9 +71,9 @@ class DataStreamOverAggregate(
 
     super.explainTerms(pw)
       .itemIf("partitionBy", partitionToString(inputType, partitionKeys), partitionKeys.nonEmpty)
-        .item("orderBy",orderingToString(inputType, overWindow.orderKeys.getFieldCollations))
-      .itemIf("rows", windowRange(overWindow), overWindow.isRows)
-      .itemIf("range", windowRange(overWindow), !overWindow.isRows)
+      .item("orderBy",orderingToString(inputType, overWindow.orderKeys.getFieldCollations))
+      .itemIf("rows", windowRange(logicWindow, overWindow, getInput), overWindow.isRows)
+      .itemIf("range", windowRange(logicWindow, overWindow, getInput), !overWindow.isRows)
       .item(
         "select", aggregationToString(
           inputType,
@@ -99,20 +100,58 @@ class DataStreamOverAggregate(
       .getFieldList
       .get(overWindow.orderKeys.getFieldCollations.get(0).getFieldIndex)
       .getValue
-
     timeType match {
       case _: ProcTimeType =>
-        // both ROWS and RANGE clause with UNBOUNDED PRECEDING and CURRENT ROW condition.
-        if (overWindow.lowerBound.isUnbounded &&
-          overWindow.upperBound.isCurrentRow) {
+        // proc-time OVER window
+        if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
+          // non-bounded OVER window
           createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS)
+        } else if (
+          overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
+              overWindow.upperBound.isCurrentRow) {
+          // bounded OVER window
+          if (overWindow.isRows) {
+            // ROWS clause bounded OVER window
+            throw new TableException(
+              "ROWS clause bounded proc-time OVER window no supported yet.")
+          } else {
+            // RANGE clause bounded OVER window
+            throw new TableException(
+              "RANGE clause bounded proc-time OVER window no supported yet.")
+          }
         } else {
           throw new TableException(
-              "OVER window only support ProcessingTime UNBOUNDED PRECEDING and CURRENT ROW " +
-              "condition.")
+            "OVER window only support ProcessingTime UNBOUNDED PRECEDING and CURRENT ROW " +
+                "condition.")
         }
       case _: RowTimeType =>
-        throw new TableException("OVER Window of the EventTime type is not currently supported.")
+        // row-time OVER window
+        if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
+          // non-bounded OVER window
+          if (overWindow.isRows) {
+            // ROWS clause unbounded OVER window
+            throw new TableException(
+              "ROWS clause unbounded row-time OVER window no supported yet.")
+          } else {
+            // RANGE clause unbounded OVER window
+            throw new TableException(
+              "RANGE clause unbounded row-time OVER window no supported yet.")
+          }
+        } else if (overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
+            overWindow.upperBound.isCurrentRow) {
+          // bounded OVER window
+          if (overWindow.isRows) {
+            // ROWS clause bounded OVER window
+            createRowsClauseBoundedAndCurrentRowOverWindow(inputDS, true)
+          } else {
+            // RANGE clause bounded OVER window
+            throw new TableException(
+              "RANGE clause bounded row-time OVER window no supported yet.")
+          }
+        } else {
+          throw new TableException(
+            "row-time OVER window only support CURRENT ROW condition.")
+        }
       case _ =>
         throw new TableException(s"Unsupported time type {$timeType}")
     }
@@ -120,7 +159,7 @@ class DataStreamOverAggregate(
   }
 
   def createUnboundedAndCurrentRowProcessingTimeOverWindow(
-    inputDS: DataStream[Row]): DataStream[Row]  = {
+    inputDS: DataStream[Row]): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
     val partitionKeys: Array[Int] = overWindow.keys.toArray
@@ -130,32 +169,78 @@ class DataStreamOverAggregate(
     val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 
     val result: DataStream[Row] =
-        // partitioned aggregation
-        if (partitionKeys.nonEmpty) {
-          val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
-            namedAggregates,
-            inputType)
+    // partitioned aggregation
+      if (partitionKeys.nonEmpty) {
+        val processFunction = AggregateUtil.createUnboundedProcessingOverProcessFunction(
+          namedAggregates,
+          inputType)
 
-          inputDS
+        inputDS
           .keyBy(partitionKeys: _*)
           .process(processFunction)
           .returns(rowTypeInfo)
           .name(aggOpName)
           .asInstanceOf[DataStream[Row]]
-        }
-        // non-partitioned aggregation
-        else {
-          val processFunction = AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
-            namedAggregates,
-            inputType,
-            false)
-
-          inputDS
-            .process(processFunction).setParallelism(1).setMaxParallelism(1)
-            .returns(rowTypeInfo)
-            .name(aggOpName)
-            .asInstanceOf[DataStream[Row]]
-        }
+      }
+      // non-partitioned aggregation
+      else {
+        val processFunction = AggregateUtil.createUnboundedProcessingOverProcessFunction(
+          namedAggregates,
+          inputType,
+          false)
+
+        inputDS
+          .process(processFunction).setParallelism(1).setMaxParallelism(1)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataStream[Row]]
+      }
+    result
+  }
+
+  def createRowsClauseBoundedAndCurrentRowOverWindow(
+    inputDS: DataStream[Row],
+    isRowTimeType: Boolean = false): DataStream[Row] = {
+
+    val overWindow: Group = logicWindow.groups.get(0)
+    val partitionKeys: Array[Int] = overWindow.keys.toArray
+    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
+    val inputFields = (0 until inputType.getFieldCount).toArray
+
+    val precedingOffset =
+      getLowerBoundary(logicWindow, overWindow, getInput()) + 1
+
+    // get the output types
+    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
+    val processFunction = AggregateUtil.createRowsClauseBoundedOverProcessFunction(
+      namedAggregates,
+      inputType,
+      inputFields,
+      precedingOffset,
+      isRowTimeType
+    )
+    val result: DataStream[Row] =
+    // partitioned aggregation
+      if (partitionKeys.nonEmpty) {
+        inputDS
+          .keyBy(partitionKeys: _*)
+          .process(processFunction)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataStream[Row]]
+      }
+      // non-partitioned aggregation
+      else {
+        inputDS
+          .keyBy(new NullByteKeySelector[Row])
+          .process(processFunction)
+          .setParallelism(1)
+          .setMaxParallelism(1)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataStream[Row]]
+      }
     result
   }
 
@@ -180,7 +265,7 @@ class DataStreamOverAggregate(
       }
     }ORDER BY: ${orderingToString(inputType, overWindow.orderKeys.getFieldCollations)}, " +
       s"${if (overWindow.isRows) "ROWS" else "RANGE"}" +
-      s"${windowRange(overWindow)}, " +
+      s"${windowRange(logicWindow, overWindow, getInput)}, " +
       s"select: (${
         aggregationToString(
           inputType,

http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index 9feec17..0084ee5 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -61,7 +61,7 @@ object AggregateUtil {
     * @param isPartitioned Flag to indicate whether the input is partitioned or not
     * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
     */
-  private[flink] def CreateUnboundedProcessingOverProcessFunction(
+  private[flink] def createUnboundedProcessingOverProcessFunction(
     namedAggregates: Seq[CalcitePair[AggregateCall, String]],
     inputType: RelDataType,
     isPartitioned: Boolean = true): ProcessFunction[Row, Row] = {
@@ -91,6 +91,52 @@ object AggregateUtil {
   }
 
   /**
+    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for ROWS clause
+    * bounded OVER window to evaluate final aggregate value.
+    *
+    * @param namedAggregates List of calls to aggregate functions and their output field names
+    * @param inputType       Input row type
+    * @param inputFields     All input fields
+    * @param precedingOffset the preceding offset
+    * @param isRowTimeType   It is a tag that indicates whether the time type is rowTimeType
+    * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
+    */
+  private[flink] def createRowsClauseBoundedOverProcessFunction(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    inputFields: Array[Int],
+    precedingOffset: Long,
+    isRowTimeType: Boolean): ProcessFunction[Row, Row] = {
+
+    val (aggFields, aggregates) =
+      transformToAggregateFunctions(
+        namedAggregates.map(_.getKey),
+        inputType,
+        needRetraction = true)
+
+    val aggregationStateType: RowTypeInfo =
+      createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
+
+    val inputRowType: RowTypeInfo =
+      createDataSetAggregateBufferDataType(inputFields, Array(), inputType)
+
+      val processFunction = if (isRowTimeType) {
+        new RowsClauseBoundedOverProcessFunction(
+          aggregates,
+          aggFields,
+          inputType.getFieldCount,
+          aggregationStateType,
+          inputRowType,
+          precedingOffset
+        )
+      } else {
+        throw TableException(
+          "Bounded partitioned proc-time OVER aggregation is not supported yet.")
+      }
+      processFunction
+  }
+
+  /**
     * Create a [[org.apache.flink.api.common.functions.MapFunction]] that prepares for aggregates.
     * The output of the function contains the grouping keys and the timestamp and the intermediate
     * aggregate values of all aggregate function. The timestamp field is aligned to time window

http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowsClauseBoundedOverProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowsClauseBoundedOverProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowsClauseBoundedOverProcessFunction.scala
new file mode 100644
index 0000000..1678d57
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowsClauseBoundedOverProcessFunction.scala
@@ -0,0 +1,239 @@
+/*
+ * 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.table.runtime.aggregate
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
+import org.apache.flink.types.Row
+import org.apache.flink.util.{Collector, Preconditions}
+
+/**
+ * Process Function for ROWS clause event-time bounded OVER window
+ *
+ * @param aggregates           the list of all [[AggregateFunction]] used for this aggregation
+ * @param aggFields            the position (in the input Row) of the input value for each aggregate
+ * @param forwardedFieldCount  the count of forwarded fields.
+ * @param aggregationStateType the row type info of aggregation
+ * @param inputRowType         the row type info of input row
+ * @param precedingOffset      the preceding offset
+ */
+class RowsClauseBoundedOverProcessFunction(
+    private val aggregates: Array[AggregateFunction[_]],
+    private val aggFields: Array[Int],
+    private val forwardedFieldCount: Int,
+    private val aggregationStateType: RowTypeInfo,
+    private val inputRowType: RowTypeInfo,
+    private val precedingOffset: Long)
+  extends ProcessFunction[Row, Row] {
+
+  Preconditions.checkNotNull(aggregates)
+  Preconditions.checkNotNull(aggFields)
+  Preconditions.checkArgument(aggregates.length == aggFields.length)
+  Preconditions.checkNotNull(forwardedFieldCount)
+  Preconditions.checkNotNull(aggregationStateType)
+  Preconditions.checkNotNull(precedingOffset)
+
+  private var output: Row = _
+
+  // the state which keeps the last triggering timestamp
+  private var lastTriggeringTsState: ValueState[Long] = _
+
+  // the state which keeps the count of data
+  private var dataCountState: ValueState[Long] = _
+
+  // the state which used to materialize the accumulator for incremental calculation
+  private var accumulatorState: ValueState[Row] = _
+
+  // the state which keeps all the data that are not expired.
+  // The first element (as the mapState key) of the tuple is the time stamp. Per each time stamp,
+  // the second element of tuple is a list that contains the entire data of all the rows belonging
+  // to this time stamp.
+  private var dataState: MapState[Long, JList[Row]] = _
+
+  override def open(config: Configuration) {
+
+    output = new Row(forwardedFieldCount + aggregates.length)
+
+    val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+      new ValueStateDescriptor[Long]("lastTriggeringTsState", classOf[Long])
+    lastTriggeringTsState = getRuntimeContext.getState(lastTriggeringTsDescriptor)
+
+    val dataCountStateDescriptor =
+      new ValueStateDescriptor[Long]("dataCountState", classOf[Long])
+    dataCountState = getRuntimeContext.getState(dataCountStateDescriptor)
+
+    val accumulatorStateDescriptor =
+      new ValueStateDescriptor[Row]("accumulatorState", aggregationStateType)
+    accumulatorState = getRuntimeContext.getState(accumulatorStateDescriptor)
+
+    val keyTypeInformation: TypeInformation[Long] =
+      BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+    val valueTypeInformation: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputRowType)
+
+    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+      new MapStateDescriptor[Long, JList[Row]](
+        "dataState",
+        keyTypeInformation,
+        valueTypeInformation)
+
+    dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+  }
+
+  override def processElement(
+    input: Row,
+    ctx: ProcessFunction[Row, Row]#Context,
+    out: Collector[Row]): Unit = {
+
+    // triggering timestamp for trigger calculation
+    val triggeringTs = ctx.timestamp
+
+    val lastTriggeringTs = lastTriggeringTsState.value
+    // check if the data is expired, if not, save the data and register event time timer
+
+    if (triggeringTs > lastTriggeringTs) {
+      val data = dataState.get(triggeringTs)
+      if (null != data) {
+        data.add(input)
+        dataState.put(triggeringTs, data)
+      } else {
+        val data = new util.ArrayList[Row]
+        data.add(input)
+        dataState.put(triggeringTs, data)
+        // register event time timer
+        ctx.timerService.registerEventTimeTimer(triggeringTs)
+      }
+    }
+  }
+
+  override def onTimer(
+    timestamp: Long,
+    ctx: ProcessFunction[Row, Row]#OnTimerContext,
+    out: Collector[Row]): Unit = {
+
+    // gets all window data from state for the calculation
+    val inputs: JList[Row] = dataState.get(timestamp)
+
+    if (null != inputs) {
+
+      var accumulators = accumulatorState.value
+      var dataCount = dataCountState.value
+
+      var retractList: JList[Row] = null
+      var retractTs: Long = Long.MaxValue
+      var retractCnt: Int = 0
+      var j = 0
+      var i = 0
+
+      while (j < inputs.size) {
+        val input = inputs.get(j)
+
+        // initialize when first run or failover recovery per key
+        if (null == accumulators) {
+          accumulators = new Row(aggregates.length)
+          i = 0
+          while (i < aggregates.length) {
+            accumulators.setField(i, aggregates(i).createAccumulator())
+            i += 1
+          }
+        }
+
+        var retractRow: Row = null
+
+        if (dataCount >= precedingOffset) {
+          if (null == retractList) {
+            // find the smallest timestamp
+            retractTs = Long.MaxValue
+            val dataTimestampIt = dataState.keys.iterator
+            while (dataTimestampIt.hasNext) {
+              val dataTs = dataTimestampIt.next
+              if (dataTs < retractTs) {
+                retractTs = dataTs
+              }
+            }
+            // get the oldest rows to retract them
+            retractList = dataState.get(retractTs)
+          }
+
+          retractRow = retractList.get(retractCnt)
+          retractCnt += 1
+
+          // remove retracted values from state
+          if (retractList.size == retractCnt) {
+            dataState.remove(retractTs)
+            retractList = null
+            retractCnt = 0
+          }
+        } else {
+          dataCount += 1
+        }
+
+        // copy forwarded fields to output row
+        i = 0
+        while (i < forwardedFieldCount) {
+          output.setField(i, input.getField(i))
+          i += 1
+        }
+
+        // retract old row from accumulators
+        if (null != retractRow) {
+          i = 0
+          while (i < aggregates.length) {
+            val accumulator = accumulators.getField(i).asInstanceOf[Accumulator]
+            aggregates(i).retract(accumulator, retractRow.getField(aggFields(i)))
+            i += 1
+          }
+        }
+
+        // accumulate current row and set aggregate in output row
+        i = 0
+        while (i < aggregates.length) {
+          val index = forwardedFieldCount + i
+          val accumulator = accumulators.getField(i).asInstanceOf[Accumulator]
+          aggregates(i).accumulate(accumulator, input.getField(aggFields(i)))
+          output.setField(index, aggregates(i).getValue(accumulator))
+          i += 1
+        }
+        j += 1
+
+        out.collect(output)
+      }
+
+      // update all states
+      if (dataState.contains(retractTs)) {
+        if (retractCnt > 0) {
+          retractList.subList(0, retractCnt).clear()
+          dataState.put(retractTs, retractList)
+        }
+      }
+      dataCountState.update(dataCount)
+      accumulatorState.update(accumulators)
+    }
+
+    lastTriggeringTsState.update(timestamp)
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index d5a140a..19350a7 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -19,14 +19,18 @@
 package org.apache.flink.table.api.scala.stream.sql
 
 import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.table.api.scala.stream.sql.SqlITCase.EventTimeSourceFunction
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.watermark.Watermark
 import org.apache.flink.table.api.{TableEnvironment, TableException}
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.scala.stream.utils.{StreamingWithStateTestBase, StreamITCase,
-StreamTestData}
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase}
 import org.apache.flink.types.Row
 import org.junit.Assert._
 import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
 
 import scala.collection.mutable
 
@@ -293,6 +297,120 @@ class SqlITCase extends StreamingWithStateTestBase {
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
+  @Test
+  def testBoundPartitionedEventTimeWindowWithRow(): Unit = {
+    val data = Seq(
+      Left((1L, (1L, 1, "Hello"))),
+      Left((2L, (2L, 2, "Hello"))),
+      Left((1L, (1L, 1, "Hello"))),
+      Left((2L, (2L, 2, "Hello"))),
+      Left((2L, (2L, 2, "Hello"))),
+      Left((1L, (1L, 1, "Hello"))),
+      Left((3L, (7L, 7, "Hello World"))),
+      Left((1L, (7L, 7, "Hello World"))),
+      Left((1L, (7L, 7, "Hello World"))),
+      Right(2L),
+      Left((3L, (3L, 3, "Hello"))),
+      Left((4L, (4L, 4, "Hello"))),
+      Left((5L, (5L, 5, "Hello"))),
+      Left((6L, (6L, 6, "Hello"))),
+      Left((20L, (20L, 20, "Hello World"))),
+      Right(6L),
+      Left((8L, (8L, 8, "Hello World"))),
+      Left((7L, (7L, 7, "Hello World"))),
+      Right(20L))
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t1 = env
+      .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val sqlQuery = "SELECT " +
+      "c, a, " +
+      "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      ", sum(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      " from T1"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3",
+      "Hello,2,3,4", "Hello,2,3,5","Hello,2,3,6",
+      "Hello,3,3,7", "Hello,4,3,9", "Hello,5,3,12",
+      "Hello,6,3,15",
+      "Hello World,7,1,7", "Hello World,7,2,14", "Hello World,7,3,21",
+      "Hello World,7,3,21", "Hello World,8,3,22", "Hello World,20,3,35")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testBoundNonPartitionedEventTimeWindowWithRow(): Unit = {
+
+    val data = Seq(
+      Left((2L, (2L, 2, "Hello"))),
+      Left((2L, (2L, 2, "Hello"))),
+      Left((1L, (1L, 1, "Hello"))),
+      Left((1L, (1L, 1, "Hello"))),
+      Left((2L, (2L, 2, "Hello"))),
+      Left((1L, (1L, 1, "Hello"))),
+      Left((20L, (20L, 20, "Hello World"))), // early row
+      Right(3L),
+      Left((2L, (2L, 2, "Hello"))), // late row
+      Left((3L, (3L, 3, "Hello"))),
+      Left((4L, (4L, 4, "Hello"))),
+      Left((5L, (5L, 5, "Hello"))),
+      Left((6L, (6L, 6, "Hello"))),
+      Left((7L, (7L, 7, "Hello World"))),
+      Right(7L),
+      Left((9L, (9L, 9, "Hello World"))),
+      Left((8L, (8L, 8, "Hello World"))),
+      Left((8L, (8L, 8, "Hello World"))),
+      Right(20L))
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    env.setParallelism(1)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t1 = env
+      .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val sqlQuery = "SELECT " +
+      "c, a, " +
+      "count(a) OVER (ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)," +
+      "sum(a) OVER (ORDER BY RowTime() ROWS BETWEEN 2 preceding AND CURRENT ROW)" +
+      "from T1"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3",
+      "Hello,2,3,4", "Hello,2,3,5", "Hello,2,3,6",
+      "Hello,3,3,7",
+      "Hello,4,3,9", "Hello,5,3,12",
+      "Hello,6,3,15", "Hello World,7,3,18",
+      "Hello World,8,3,21", "Hello World,8,3,23",
+      "Hello World,9,3,25",
+      "Hello World,20,3,37")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
   /**
     *  All aggregates must be computed on the same window.
     */
@@ -317,4 +435,21 @@ class SqlITCase extends StreamingWithStateTestBase {
     result.addSink(new StreamITCase.StringSink)
     env.execute()
   }
+
+}
+
+object SqlITCase {
+
+  class EventTimeSourceFunction[T](
+      dataWithTimestampList: Seq[Either[(Long, T), Long]]) extends SourceFunction[T] {
+    override def run(ctx: SourceContext[T]): Unit = {
+      dataWithTimestampList.foreach {
+        case Left(t) => ctx.collectWithTimestamp(t._2, t._1)
+        case Right(w) => ctx.emitWatermark(new Watermark(w))
+      }
+    }
+
+    override def cancel(): Unit = ???
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a9d39fe/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index a25e59c..9a425b3 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -239,4 +239,59 @@ class WindowAggregateTest extends TableTestBase {
       )
     streamUtil.verifySql(sql, expected)
   }
+
+  @Test
+  def testBoundPartitionedRowTimeWindowWithRow() = {
+    val sql = "SELECT " +
+        "c, " +
+        "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
+        "CURRENT ROW) as cnt1 " +
+        "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("partitionBy", "c"),
+          term("orderBy", "ROWTIME"),
+          term("rows", "BETWEEN 5 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
+  @Test
+  def testBoundNonPartitionedRowTimeWindowWithRow() = {
+    val sql = "SELECT " +
+        "c, " +
+        "count(a) OVER (ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
+        "CURRENT ROW) as cnt1 " +
+        "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("orderBy", "ROWTIME"),
+          term("rows", "BETWEEN 5 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
 }


[03/50] [abbrv] flink git commit: [FLINK-6170] [metrics] Don't rely on stats snapshot for checkpoint metrics

Posted by fh...@apache.org.
[FLINK-6170] [metrics] Don't rely on stats snapshot for checkpoint metrics

This closes #3597.


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

Branch: refs/heads/table-retraction
Commit: d0695c054737f18ade0d5c5d95e56202d041fc60
Parents: 64c7b11
Author: Ufuk Celebi <uc...@apache.org>
Authored: Wed Mar 22 18:08:13 2017 +0100
Committer: Ufuk Celebi <uc...@apache.org>
Committed: Wed Mar 22 21:20:56 2017 +0100

----------------------------------------------------------------------
 .../checkpoint/CheckpointStatsTracker.java      |  30 +--
 .../checkpoint/CheckpointStatsTrackerTest.java  | 232 +++++++++++++++++--
 2 files changed, 235 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d0695c05/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
index d324c25..c7efb7b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
@@ -18,6 +18,13 @@
 
 package org.apache.flink.runtime.checkpoint;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.annotation.Nullable;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.metrics.Metric;
@@ -26,14 +33,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
 
-import javax.annotation.Nullable;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
 /**
  * Tracker for checkpoint statistics.
  *
@@ -96,6 +95,10 @@ public class CheckpointStatsTracker {
 	 */
 	private volatile boolean dirty;
 
+	/** The latest completed checkpoint. Used by the latest completed checkpoint metrics. */
+	@Nullable
+	private volatile transient CompletedCheckpointStats latestCompletedCheckpoint;
+
 	/**
 	 * Creates a new checkpoint stats tracker.
 	 *
@@ -241,6 +244,8 @@ public class CheckpointStatsTracker {
 	private void reportCompletedCheckpoint(CompletedCheckpointStats completed) {
 		statsReadWriteLock.lock();
 		try {
+			latestCompletedCheckpoint = completed;
+
 			counts.incrementCompletedCheckpoints();
 			history.replacePendingCheckpointById(completed);
 
@@ -400,7 +405,7 @@ public class CheckpointStatsTracker {
 	private class LatestCompletedCheckpointSizeGauge implements Gauge<Long> {
 		@Override
 		public Long getValue() {
-			CompletedCheckpointStats completed = latestSnapshot.getHistory().getLatestCompletedCheckpoint();
+			CompletedCheckpointStats completed = latestCompletedCheckpoint;
 			if (completed != null) {
 				return completed.getStateSize();
 			} else {
@@ -412,7 +417,7 @@ public class CheckpointStatsTracker {
 	private class LatestCompletedCheckpointDurationGauge implements Gauge<Long> {
 		@Override
 		public Long getValue() {
-			CompletedCheckpointStats completed = latestSnapshot.getHistory().getLatestCompletedCheckpoint();
+			CompletedCheckpointStats completed = latestCompletedCheckpoint;
 			if (completed != null) {
 				return completed.getEndToEndDuration();
 			} else {
@@ -421,11 +426,10 @@ public class CheckpointStatsTracker {
 		}
 	}
 
-
 	private class LatestCompletedCheckpointAlignmentBufferedGauge implements Gauge<Long> {
 		@Override
 		public Long getValue() {
-			CompletedCheckpointStats completed = latestSnapshot.getHistory().getLatestCompletedCheckpoint();
+			CompletedCheckpointStats completed = latestCompletedCheckpoint;;
 			if (completed != null) {
 				return completed.getAlignmentBuffered();
 			} else {
@@ -437,7 +441,7 @@ public class CheckpointStatsTracker {
 	private class LatestCompletedCheckpointExternalPathGauge implements Gauge<String> {
 		@Override
 		public String getValue() {
-			CompletedCheckpointStats completed = latestSnapshot.getHistory().getLatestCompletedCheckpoint();
+			CompletedCheckpointStats completed = latestCompletedCheckpoint;
 			if (completed != null) {
 				return completed.getExternalPath();
 			} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/d0695c05/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
index 7ab71cb..aaf1774 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java
@@ -18,18 +18,6 @@
 
 package org.apache.flink.runtime.checkpoint;
 
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
-import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.Iterator;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -42,6 +30,23 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import org.apache.flink.metrics.CharacterFilter;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Histogram;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
+import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
+import org.junit.Test;
+
 public class CheckpointStatsTrackerTest {
 
 	/**
@@ -275,10 +280,10 @@ public class CheckpointStatsTrackerTest {
 	}
 
 	/**
-	 * Tests the registered metrics.
+	 * Tests the registration of the checkpoint metrics.
 	 */
 	@Test
-	public void testMetrics() throws Exception {
+	public void testMetricsRegistration() throws Exception {
 		MetricGroup metricGroup = mock(MetricGroup.class);
 
 		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
@@ -305,6 +310,205 @@ public class CheckpointStatsTrackerTest {
 		verify(metricGroup, times(9)).gauge(any(String.class), any(Gauge.class));
 	}
 
+	/**
+	 * Tests that the metrics are updated properly. We had a bug that required new stats
+	 * snapshots in order to update the metrics.
+	 */
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testMetricsAreUpdated() throws Exception {
+		final Map<String, Gauge<?>> registeredGauges = new HashMap<>();
+
+		MetricGroup metricGroup = new MetricGroup() {
+			@Override
+			public Counter counter(int name) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public Counter counter(String name) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <C extends Counter> C counter(int name, C counter) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <C extends Counter> C counter(String name, C counter) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <T, G extends Gauge<T>> G gauge(int name, G gauge) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <T, G extends Gauge<T>> G gauge(String name, G gauge) {
+				registeredGauges.put(name, gauge);
+				return gauge;
+			}
+
+			@Override
+			public <H extends Histogram> H histogram(String name, H histogram) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <H extends Histogram> H histogram(int name, H histogram) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <M extends Meter> M meter(String name, M meter) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public <M extends Meter> M meter(int name, M meter) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public MetricGroup addGroup(int name) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public MetricGroup addGroup(String name) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public String[] getScopeComponents() {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public Map<String, String> getAllVariables() {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public String getMetricIdentifier(String metricName) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+
+			@Override
+			public String getMetricIdentifier(String metricName, CharacterFilter filter) {
+				throw new UnsupportedOperationException("Not expected in this test");
+			}
+		};
+
+		ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class);
+		when(jobVertex.getJobVertexId()).thenReturn(new JobVertexID());
+		when(jobVertex.getParallelism()).thenReturn(1);
+
+		CheckpointStatsTracker stats = new CheckpointStatsTracker(
+			0,
+			Collections.singletonList(jobVertex),
+			mock(JobSnapshottingSettings.class),
+			metricGroup);
+
+		// Make sure to adjust this test if metrics are added/removed
+		assertEquals(9, registeredGauges.size());
+
+		// Check initial values
+		Gauge<Long> numCheckpoints = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_CHECKPOINTS_METRIC);
+		Gauge<Integer> numInProgressCheckpoints = (Gauge<Integer>) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_IN_PROGRESS_CHECKPOINTS_METRIC);
+		Gauge<Long> numCompletedCheckpoints = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_COMPLETED_CHECKPOINTS_METRIC);
+		Gauge<Long> numFailedCheckpoints = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_FAILED_CHECKPOINTS_METRIC);
+		Gauge<Long> latestRestoreTimestamp = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC);
+		Gauge<Long> latestCompletedSize = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC);
+		Gauge<Long> latestCompletedDuration = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC);
+		Gauge<Long> latestCompletedAlignmentBuffered = (Gauge<Long>) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC);
+		Gauge<String> latestCompletedExternalPath = (Gauge<String>) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC);
+
+		assertEquals(Long.valueOf(0), numCheckpoints.getValue());
+		assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue());
+		assertEquals(Long.valueOf(0), numCompletedCheckpoints.getValue());
+		assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue());
+		assertEquals(Long.valueOf(-1), latestRestoreTimestamp.getValue());
+		assertEquals(Long.valueOf(-1), latestCompletedSize.getValue());
+		assertEquals(Long.valueOf(-1), latestCompletedDuration.getValue());
+		assertEquals(Long.valueOf(-1), latestCompletedAlignmentBuffered.getValue());
+		assertEquals("n/a", latestCompletedExternalPath.getValue());
+
+		PendingCheckpointStats pending = stats.reportPendingCheckpoint(
+			0,
+			0,
+			CheckpointProperties.forStandardCheckpoint());
+
+		// Check counts
+		assertEquals(Long.valueOf(1), numCheckpoints.getValue());
+		assertEquals(Integer.valueOf(1), numInProgressCheckpoints.getValue());
+		assertEquals(Long.valueOf(0), numCompletedCheckpoints.getValue());
+		assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue());
+
+		long ackTimestamp = 11231230L;
+		long stateSize = 12381238L;
+		long ignored = 0;
+		long alignmenetBuffered = 182812L;
+		String externalPath = "myexternalpath";
+
+		SubtaskStateStats subtaskStats = new SubtaskStateStats(
+			0,
+			ackTimestamp,
+			stateSize,
+			ignored,
+			ignored,
+			alignmenetBuffered,
+			ignored);
+
+		assertTrue(pending.reportSubtaskStats(jobVertex.getJobVertexId(), subtaskStats));
+
+		pending.reportCompletedCheckpoint(externalPath);
+
+		// Verify completed checkpoint updated
+		assertEquals(Long.valueOf(1), numCheckpoints.getValue());
+		assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue());
+		assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue());
+		assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue());
+		assertEquals(Long.valueOf(-1), latestRestoreTimestamp.getValue());
+		assertEquals(Long.valueOf(stateSize), latestCompletedSize.getValue());
+		assertEquals(Long.valueOf(ackTimestamp), latestCompletedDuration.getValue());
+		assertEquals(Long.valueOf(alignmenetBuffered), latestCompletedAlignmentBuffered.getValue());
+		assertEquals(externalPath, latestCompletedExternalPath.getValue());
+
+		// Check failed
+		PendingCheckpointStats nextPending = stats.reportPendingCheckpoint(
+			1,
+			11,
+			CheckpointProperties.forStandardCheckpoint());
+
+		long failureTimestamp = 1230123L;
+		nextPending.reportFailedCheckpoint(failureTimestamp, null);
+
+		// Verify updated
+		assertEquals(Long.valueOf(2), numCheckpoints.getValue());
+		assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue());
+		assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue());
+		assertEquals(Long.valueOf(1), numFailedCheckpoints.getValue()); // one failed now
+
+		// Check restore
+		long restoreTimestamp = 183419283L;
+		RestoredCheckpointStats restored = new RestoredCheckpointStats(
+			1,
+			CheckpointProperties.forStandardCheckpoint(),
+			restoreTimestamp,
+			null);
+		stats.reportRestoredCheckpoint(restored);
+
+		assertEquals(Long.valueOf(2), numCheckpoints.getValue());
+		assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue());
+		assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue());
+		assertEquals(Long.valueOf(1), numFailedCheckpoints.getValue());
+
+		assertEquals(Long.valueOf(restoreTimestamp), latestRestoreTimestamp.getValue());
+	}
+
 	// ------------------------------------------------------------------------
 
 	/**


[13/50] [abbrv] flink git commit: [FLINK-5972] Don't allow shrinking merging windows

Posted by fh...@apache.org.
[FLINK-5972] Don't allow shrinking merging windows

This closes #3587.


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

Branch: refs/heads/table-retraction
Commit: 68289b1a52db7157d23085850ec947e78e729f01
Parents: 25d52e4
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Mar 21 14:58:45 2017 +0100
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Thu Mar 23 23:29:02 2017 +0800

----------------------------------------------------------------------
 .../windowing/EvictingWindowOperator.java       | 13 ++++
 .../operators/windowing/WindowOperator.java     | 15 +++-
 .../windowing/WindowOperatorContractTest.java   | 80 ++++++++++++++++++++
 3 files changed, 107 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/68289b1a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 951f661..24c8d32 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
@@ -121,6 +121,19 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 							public void merge(W mergeResult,
 									Collection<W> mergedWindows, W stateWindowResult,
 									Collection<W> mergedStateWindows) throws Exception {
+
+								if ((windowAssigner.isEventTime() && mergeResult.maxTimestamp() + allowedLateness <= internalTimerService.currentWatermark())) {
+									throw new UnsupportedOperationException("The end timestamp of an " +
+											"event-time window cannot become earlier than the current watermark " +
+											"by merging. Current watermark: " + internalTimerService.currentWatermark() +
+											" window: " + mergeResult);
+								} else if (!windowAssigner.isEventTime() && mergeResult.maxTimestamp() <= internalTimerService.currentProcessingTime()) {
+									throw new UnsupportedOperationException("The end timestamp of a " +
+											"processing-time window cannot become earlier than the current processing time " +
+											"by merging. Current processing time: " + internalTimerService.currentProcessingTime() +
+											" window: " + mergeResult);
+								}
+
 								context.key = key;
 								context.window = mergeResult;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/68289b1a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index b4283d8..3745659 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
@@ -131,7 +131,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 *         {@code window.maxTimestamp + allowedLateness} landmark.
 	 * </ul>
 	 */
-	private final long allowedLateness;
+	protected final long allowedLateness;
 
 	/**
 	 * {@link OutputTag} to use for late arriving events. Elements for which
@@ -352,6 +352,19 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					public void merge(W mergeResult,
 							Collection<W> mergedWindows, W stateWindowResult,
 							Collection<W> mergedStateWindows) throws Exception {
+
+						if ((windowAssigner.isEventTime() && mergeResult.maxTimestamp() + allowedLateness <= internalTimerService.currentWatermark())) {
+							throw new UnsupportedOperationException("The end timestamp of an " +
+									"event-time window cannot become earlier than the current watermark " +
+									"by merging. Current watermark: " + internalTimerService.currentWatermark() +
+									" window: " + mergeResult);
+						} else if (!windowAssigner.isEventTime() && mergeResult.maxTimestamp() <= internalTimerService.currentProcessingTime()) {
+							throw new UnsupportedOperationException("The end timestamp of a " +
+									"processing-time window cannot become earlier than the current processing time " +
+									"by merging. Current processing time: " + internalTimerService.currentProcessingTime() +
+									" window: " + mergeResult);
+						}
+
 						context.key = key;
 						context.window = mergeResult;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/68289b1a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
index aaea8b1..8aae46a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -1479,6 +1479,86 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	}
 
 	@Test
+	public void testRejectShrinkingMergingEventTimeWindows() throws Exception {
+		testRejectShrinkingMergingWindows(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testRejectShrinkingMergingProcessingTimeWindows() throws Exception {
+		testRejectShrinkingMergingWindows(new ProcessingTimeAdaptor());
+	}
+
+	/**
+	 * A misbehaving {@code WindowAssigner} can cause a window to become late by merging if
+	 * it moves the end-of-window time before the watermark. This verifies that we don't allow that.
+	 */
+	void testRejectShrinkingMergingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
+		int allowedLateness = 10;
+
+		if (timeAdaptor instanceof ProcessingTimeAdaptor) {
+			// we don't have allowed lateness for processing time
+			allowedLateness = 0;
+		}
+
+		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, allowedLateness, mockWindowFunction);
+
+		testHarness.open();
+
+		timeAdaptor.advanceTime(testHarness, 0);
+
+		assertEquals(0, testHarness.extractOutputStreamRecords().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 22)));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(2, testHarness.numKeyedStateEntries()); // window contents and merging window set
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // cleanup timer
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 25)));
+
+		timeAdaptor.advanceTime(testHarness, 20);
+
+		// our window should still be there
+		assertEquals(2, testHarness.numKeyedStateEntries()); // window contents and merging window set
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // cleanup timer
+
+		// the result timestamp is ... + 2 because a watermark t says no element with
+		// timestamp <= t will come in the future and because window ends are exclusive:
+		// a window (0, 12) will have 11 as maxTimestamp. With the watermark at 20, 10 would
+		// already be considered late
+		shouldMergeWindows(
+				mockAssigner,
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 22), new TimeWindow(0, 25))),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 22), new TimeWindow(0, 25))),
+				new TimeWindow(0, 20 - allowedLateness + 2));
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		// now merge it to a window that is just late
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(0, 25)));
+
+		shouldMergeWindows(
+				mockAssigner,
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 20 - allowedLateness + 2), new TimeWindow(0, 25))),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 20 - allowedLateness + 2), new TimeWindow(0, 25))),
+				new TimeWindow(0, 20 - allowedLateness + 1));
+
+		expectedException.expect(UnsupportedOperationException.class);
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+	}
+
+	@Test
 	public void testMergingOfExistingEventTimeWindows() throws Exception {
 		testMergingOfExistingWindows(new EventTimeAdaptor());
 	}


[41/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Small code cleanups in ExecutionGraph and related classes

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Small code cleanups in ExecutionGraph and related classes


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

Branch: refs/heads/table-retraction
Commit: 874d956561f817a2578d7d7e6686d598323dc4c8
Parents: 69843fe
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Mar 21 18:12:23 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 17:11:49 2017 +0200

----------------------------------------------------------------------
 .../flink/runtime/executiongraph/Execution.java |   6 +-
 .../runtime/executiongraph/ExecutionGraph.java  | 140 +++++++++----------
 .../flink/runtime/executiongraph/IOMetrics.java |   2 +
 .../runtime/executiongraph/JobInformation.java  |   8 ++
 .../runtime/taskmanager/TaskExecutionState.java |   3 +-
 ...ExecutionGraphCheckpointCoordinatorTest.java |  12 +-
 .../ExecutionGraphMetricsTest.java              |   2 +-
 7 files changed, 87 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index e17a3e5..1a3ef11 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -820,7 +820,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 	void sendPartitionInfos() {
 		// check if the ExecutionVertex has already been archived and thus cleared the
 		// partial partition infos queue
-		if(partialInputChannelDeploymentDescriptors != null && !partialInputChannelDeploymentDescriptors.isEmpty()) {
+		if (partialInputChannelDeploymentDescriptors != null && !partialInputChannelDeploymentDescriptors.isEmpty()) {
 
 			PartialInputChannelDeploymentDescriptor partialInputChannelDeploymentDescriptor;
 
@@ -931,7 +931,9 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 			}
 			else if (currentState == CANCELING || currentState == FAILED) {
 				if (LOG.isDebugEnabled()) {
-					LOG.debug(String.format("Concurrent canceling/failing of %s while deployment was in progress.", getVertexWithAttempt()));
+					// this log statement is guarded because the 'getVertexWithAttempt()' method
+					// performs string concatenations 
+					LOG.debug("Concurrent canceling/failing of {} while deployment was in progress.", getVertexWithAttempt());
 				}
 				sendCancelRpcCall();
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 6bb3455..e911f49 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.executiongraph;
 
 import org.apache.commons.lang3.StringUtils;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.Archiveable;
 import org.apache.flink.api.common.ArchivedExecutionConfig;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -63,7 +64,6 @@ import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableExceptio
 import org.apache.flink.runtime.query.KvStateLocationRegistry;
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.runtime.util.SerializedThrowable;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
@@ -92,6 +92,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
 
@@ -131,7 +132,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 	/** The lock used to secure all access to mutable fields, especially the tracking of progress
 	 * within the job. */
-	private final SerializableObject progressLock = new SerializableObject();
+	private final Object progressLock = new Object();
 
 	/** Job specific information like the job id, job name, job configuration, etc. */
 	private final JobInformation jobInformation;
@@ -222,7 +223,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 	/** Checkpoint stats tracker separate from the coordinator in order to be
 	 * available after archiving. */
-	@SuppressWarnings("NonSerializableFieldInSerializableClass")
 	private CheckpointStatsTracker checkpointStatsTracker;
 
 	// ------ Fields that are only relevant for archived execution graphs ------------
@@ -235,6 +235,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	/**
 	 * This constructor is for tests only, because it does not include class loading information.
 	 */
+	@VisibleForTesting
 	ExecutionGraph(
 			ScheduledExecutorService futureExecutor,
 			Executor ioExecutor,
@@ -369,24 +370,16 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			CheckpointStatsTracker statsTracker) {
 
 		// simple sanity checks
-		if (interval < 10 || checkpointTimeout < 10) {
-			throw new IllegalArgumentException();
-		}
-		if (state != JobStatus.CREATED) {
-			throw new IllegalStateException("Job must be in CREATED state");
-		}
+		checkArgument(interval >= 10, "checkpoint interval must not be below 10ms");
+		checkArgument(checkpointTimeout >= 10, "checkpoint timeout must not be below 10ms");
+
+		checkState(state == JobStatus.CREATED, "Job must be in CREATED state");
+		checkState(checkpointCoordinator == null, "checkpointing already enabled");
 
 		ExecutionVertex[] tasksToTrigger = collectExecutionVertices(verticesToTrigger);
 		ExecutionVertex[] tasksToWaitFor = collectExecutionVertices(verticesToWaitFor);
 		ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo);
 
-		// disable to make sure existing checkpoint coordinators are cleared
-		try {
-			disableSnaphotCheckpointing();
-		} catch (Throwable t) {
-			LOG.error("Error while shutting down checkpointer.");
-		}
-
 		checkpointStatsTracker = checkNotNull(statsTracker, "CheckpointStatsTracker");
 
 		// create the coordinator that triggers and commits checkpoints and holds the state
@@ -416,24 +409,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		}
 	}
 
-	/**
-	 * Disables checkpointing.
-	 *
-	 * <p>The shutdown of the checkpoint coordinator might block. Make sure that calls to this
-	 * method don't block the job manager actor and run asynchronously.
-	 */
-	public void disableSnaphotCheckpointing() throws Exception {
-		if (state != JobStatus.CREATED) {
-			throw new IllegalStateException("Job must be in CREATED state");
-		}
-
-		if (checkpointCoordinator != null) {
-			checkpointCoordinator.shutdown(state);
-			checkpointCoordinator = null;
-			checkpointStatsTracker = null;
-		}
-	}
-
 	@Override
 	public CheckpointCoordinator getCheckpointCoordinator() {
 		return checkpointCoordinator;
@@ -761,7 +736,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 	private void scheduleLazy(SlotProvider slotProvider) throws NoResourceAvailableException {
 		// simply take the vertices without inputs.
-		for (ExecutionJobVertex ejv : this.tasks.values()) {
+		for (ExecutionJobVertex ejv : verticesInCreationOrder) {
 			if (ejv.getJobVertex().isInputVertex()) {
 				ejv.scheduleAll(slotProvider, allowQueuedScheduling);
 			}
@@ -932,9 +907,9 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	}
 
 	public void stop() throws StoppingException {
-		if(this.isStoppable) {
-			for(ExecutionVertex ev : this.getAllExecutionVertices()) {
-				if(ev.getNumberOfInputs() == 0) { // send signal to sources only
+		if (isStoppable) {
+			for (ExecutionVertex ev : this.getAllExecutionVertices()) {
+				if (ev.getNumberOfInputs() == 0) { // send signal to sources only
 					ev.stop();
 				}
 			}
@@ -1011,7 +986,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 				return;
 			}
 
-			// no need to treat other states
+			// else: concurrent change to execution state, retry
 		}
 	}
 
@@ -1273,35 +1248,47 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	 * @return True, if the task update was properly applied, false, if the execution attempt was not found.
 	 */
 	public boolean updateState(TaskExecutionState state) {
-		Execution attempt = this.currentExecutions.get(state.getID());
+		final Execution attempt = currentExecutions.get(state.getID());
+
 		if (attempt != null) {
+			try {
+				Map<String, Accumulator<?, ?>> accumulators;
+
+				switch (state.getExecutionState()) {
+					case RUNNING:
+						return attempt.switchToRunning();
+	
+					case FINISHED:
+						// this deserialization is exception-free
+						accumulators = deserializeAccumulators(state);
+						attempt.markFinished(accumulators, state.getIOMetrics());
+						return true;
+	
+					case CANCELED:
+						// this deserialization is exception-free
+						accumulators = deserializeAccumulators(state);
+						attempt.cancelingComplete(accumulators, state.getIOMetrics());
+						return true;
+	
+					case FAILED:
+						// this deserialization is exception-free
+						accumulators = deserializeAccumulators(state);
+						attempt.markFailed(state.getError(userClassLoader), accumulators, state.getIOMetrics());
+						return true;
+	
+					default:
+						// we mark as failed and return false, which triggers the TaskManager
+						// to remove the task
+						attempt.fail(new Exception("TaskManager sent illegal state update: " + state.getExecutionState()));
+						return false;
+				}
+			}
+			catch (Throwable t) {
+				ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
 
-			switch (state.getExecutionState()) {
-				case RUNNING:
-					return attempt.switchToRunning();
-				case FINISHED:
-					try {
-						Map<String, Accumulator<?, ?>> userAccumulators = deserializeAccumulators(state);
-						attempt.markFinished(userAccumulators, state.getIOMetrics());
-					}
-					catch (Exception e) {
-						LOG.error("Failed to deserialize final accumulator results.", e);
-						attempt.markFailed(e);
-					}
-					return true;
-				case CANCELED:
-					Map<String, Accumulator<?, ?>> userAcc1 = deserializeAccumulators(state);
-					attempt.cancelingComplete(userAcc1, state.getIOMetrics());
-					return true;
-				case FAILED:
-					Map<String, Accumulator<?, ?>> userAcc2 = deserializeAccumulators(state);
-					attempt.markFailed(state.getError(userClassLoader), userAcc2, state.getIOMetrics());
-					return true;
-				default:
-					// we mark as failed and return false, which triggers the TaskManager
-					// to remove the task
-					attempt.fail(new Exception("TaskManager sent illegal state update: " + state.getExecutionState()));
-					return false;
+				// failures during updates leave the ExecutionGraph inconsistent
+				fail(t);
+				return false;
 			}
 		}
 		else {
@@ -1309,17 +1296,28 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		}
 	}
 
+	/**
+	 * Deserializes accumulators from a task state update.
+	 * 
+	 * <p>This method never throws an exception!
+	 * 
+	 * @param state The task execution state from which to deserialize the accumulators.
+	 * @return The deserialized accumulators, of null, if there are no accumulators or an error occurred.
+	 */
 	private Map<String, Accumulator<?, ?>> deserializeAccumulators(TaskExecutionState state) {
 		AccumulatorSnapshot serializedAccumulators = state.getAccumulators();
-		Map<String, Accumulator<?, ?>> accumulators = null;
+
 		if (serializedAccumulators != null) {
 			try {
-				accumulators = serializedAccumulators.deserializeUserAccumulators(userClassLoader);
-			} catch (Exception e) {
-				LOG.error("Failed to deserialize final accumulator results.", e);
+				return serializedAccumulators.deserializeUserAccumulators(userClassLoader);
+			}
+			catch (Throwable t) {
+				// we catch Throwable here to include all form of linking errors that may
+				// occur if user classes are missing in the classpath
+				LOG.error("Failed to deserialize final accumulator results.", t);
 			}
 		}
-		return accumulators;
+		return null;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java
index 82c376e..668418d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java
@@ -25,7 +25,9 @@ import java.io.Serializable;
  * An instance of this class represents a snapshot of the io-related metrics of a single task.
  */
 public class IOMetrics implements Serializable {
+
 	private static final long serialVersionUID = -7208093607556457183L;
+
 	protected long numRecordsIn;
 	protected long numRecordsOut;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
index 6e3c1e8..f497f8c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
@@ -93,4 +93,12 @@ public class JobInformation implements Serializable {
 	public Collection<URL> getRequiredClasspathURLs() {
 		return requiredClasspathURLs;
 	}
+
+	// ------------------------------------------------------------------------
+
+
+	@Override
+	public String toString() {
+		return "JobInformation for '" + jobName + "' (" + jobId + ')';
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
index 5cc2484..9395435 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
@@ -49,8 +49,9 @@ public class TaskExecutionState implements Serializable {
 
 	private final SerializedThrowable throwable;
 
-	/** Serialized flink and user-defined accumulators */
+	/** Serialized user-defined accumulators */
 	private final AccumulatorSnapshot accumulators;
+
 	private final IOMetrics ioMetrics;
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
index 98b4c4d..81162b6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
@@ -18,14 +18,11 @@
 
 package org.apache.flink.runtime.checkpoint;
 
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
-import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -37,7 +34,7 @@ import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.util.SerializedValue;
-import org.junit.AfterClass;
+
 import org.junit.Test;
 import org.mockito.Matchers;
 
@@ -50,13 +47,6 @@ import static org.mockito.Mockito.verify;
 
 public class ExecutionGraphCheckpointCoordinatorTest {
 
-	private static ActorSystem system = AkkaUtils.createLocalActorSystem(new Configuration());
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
 	/**
 	 * Tests that a shut down checkpoint coordinator calls shutdown on
 	 * the store and counter.

http://git-wip-us.apache.org/repos/asf/flink/blob/874d9565/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
index 203c547..5496e35 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
@@ -162,7 +162,7 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 				testingRestartStrategy,
 				Collections.<BlobKey>emptyList(),
 				Collections.<URL>emptyList(),
-			scheduler,
+				scheduler,
 				getClass().getClassLoader(),
 				metricGroup);
 	


[11/50] [abbrv] flink git commit: [hotfix] Fix various small issues in WindowOperatorContractTest

Posted by fh...@apache.org.
[hotfix] Fix various small issues in WindowOperatorContractTest


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

Branch: refs/heads/table-retraction
Commit: 25d52e4df216dc54d2d82e1f0b449871bda4ba74
Parents: 3c4b156
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Mar 22 17:02:15 2017 +0100
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Thu Mar 23 23:29:01 2017 +0800

----------------------------------------------------------------------
 .../windowing/WindowOperatorContractTest.java   | 97 +++++++++-----------
 1 file changed, 44 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/25d52e4d/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
index abc7b3e..aaea8b1 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 
 import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.isStreamRecord;
+import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -36,7 +37,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import com.google.common.collect.Lists;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -155,7 +156,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 	@SuppressWarnings("unchecked")
 	static Iterable<Integer> intIterable(Integer... values) {
-		return (Iterable<Integer>) argThat(containsInAnyOrder(values));
+		return (Iterable<Integer>) argThat(contains(values));
 	}
 
 	static TimeWindow anyTimeWindow() {
@@ -247,55 +248,55 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	}
 
 	private static <T> void shouldContinueOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
 	}
 
 	private static <T> void shouldFireOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
 	}
 
 	private static <T> void shouldPurgeOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
 	}
 
 	private static <T> void shouldFireAndPurgeOnElement(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		when(mockTrigger.onElement(Matchers.<T>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
 	}
 
 	private static <T> void shouldContinueOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
 	}
 
 	private static <T> void shouldFireOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
 	}
 
 	private static <T> void shouldPurgeOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
 	}
 
 	private static <T> void shouldFireAndPurgeOnEventTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onEventTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		when(mockTrigger.onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
 	}
 
 	private static <T> void shouldContinueOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.CONTINUE);
 	}
 
 	private static <T> void shouldFireOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
 	}
 
 	private static <T> void shouldPurgeOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.PURGE);
 	}
 
 	private static <T> void shouldFireAndPurgeOnProcessingTime(Trigger<T, TimeWindow> mockTrigger) throws Exception {
-		when(mockTrigger.onProcessingTime(anyLong(), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
+		when(mockTrigger.onProcessingTime(anyLong(), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE_AND_PURGE);
 	}
 
 	/**
-	 * Verify that there is no late-date side output if the {@code WindowAssigner} does
+	 * Verify that there is no late-data side output if the {@code WindowAssigner} does
 	 * not assign any windows.
 	 */
 	@Test
@@ -346,7 +347,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		verify(mockAssigner, times(1)).assignWindows(eq(0), eq(5L), anyAssignerContext());
 
 		assertThat(testHarness.getSideOutput(lateOutputTag),
-				containsInAnyOrder(isStreamRecord(0, 5L)));
+				contains(isStreamRecord(0, 5L)));
 
 		// we should also see side output if the WindowAssigner assigns no windows
 		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
@@ -358,7 +359,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		verify(mockAssigner, times(1)).assignWindows(eq(0), eq(10L), anyAssignerContext());
 
 		assertThat(testHarness.getSideOutput(lateOutputTag),
-				containsInAnyOrder(isStreamRecord(0, 5L), isStreamRecord(0, 10L)));
+				contains(isStreamRecord(0, 5L), isStreamRecord(0, 10L)));
 
 	}
 
@@ -520,7 +521,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		assertThat(testHarness.extractOutputStreamRecords(),
-				containsInAnyOrder(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
+				contains(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
 	}
 
 	@Test
@@ -534,7 +535,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	}
 
 
-	private  void testEmittingFromWindowFunction(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testEmittingFromWindowFunction(TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -573,7 +574,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		assertThat(testHarness.extractOutputStreamRecords(),
-				containsInAnyOrder(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
+				contains(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
 	}
 
 	@Test
@@ -1067,9 +1068,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 	/**
 	 * Verify that we neither invoke the trigger nor the window function if a timer
-	 * for an empty merging window.
+	 * for an empty merging window fires.
 	 */
-	public void testNoTimerFiringForPurgedMergingWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testNoTimerFiringForPurgedMergingWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1133,7 +1134,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	 * Verify that we neither invoke the trigger nor the window function if a timer
 	 * fires for a merging window that was already garbage collected.
 	 */
-	public void testNoTimerFiringForGarbageCollectedMergingWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testNoTimerFiringForGarbageCollectedMergingWindow(final TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1166,7 +1167,6 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 			}
 		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
 
-
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
 		assertEquals(2, testHarness.numKeyedStateEntries()); // window contents and merging window set
@@ -1311,7 +1311,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testDeletedTimerDoesNotFire(new ProcessingTimeAdaptor());
 	}
 
-	public void testDeletedTimerDoesNotFire(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testDeletedTimerDoesNotFire(TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1372,8 +1372,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockAssigner).mergeWindows(eq(Lists.newArrayList(new TimeWindow(2, 4))), anyMergeCallback());
-		verify(mockAssigner).mergeWindows(eq(Lists.newArrayList(new TimeWindow(2, 4), new TimeWindow(0, 2))), anyMergeCallback());
+		verify(mockAssigner).mergeWindows(eq(Collections.singletonList(new TimeWindow(2, 4))), anyMergeCallback());
+		verify(mockAssigner).mergeWindows(eq(Collections.singletonList(new TimeWindow(2, 4))), anyMergeCallback());
+
 		verify(mockAssigner, times(2)).mergeWindows(anyCollection(), anyMergeCallback());
 
 
@@ -1392,7 +1393,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	/**
 	 * Verify that windows are merged eagerly, if possible.
 	 */
-	public void testWindowsAreMergedEagerly(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testWindowsAreMergedEagerly(final TimeDomainAdaptor timeAdaptor) throws Exception {
 		// in this test we only have one state window and windows are eagerly
 		// merged into the first window
 
@@ -1456,8 +1457,8 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		shouldMergeWindows(
 				mockAssigner,
-				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)),
-				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4)),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 2), new TimeWindow(2, 4))),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 2), new TimeWindow(2, 4))),
 				new TimeWindow(0, 4));
 
 		// don't register a timer or update state in onElement, this checks
@@ -1491,7 +1492,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	 * Verify that we only keep one of the underlying state windows. This test also verifies that
 	 * GC timers are correctly deleted when merging windows.
 	 */
-	public void testMergingOfExistingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testMergingOfExistingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1562,8 +1563,8 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		shouldMergeWindows(
 				mockAssigner,
-				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3)),
-				Lists.newArrayList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3)),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3))),
+				new ArrayList<>(Arrays.asList(new TimeWindow(0, 2), new TimeWindow(2, 4), new TimeWindow(1, 3))),
 				new TimeWindow(0, 4));
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
@@ -1618,7 +1619,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testOnTimePurgeDoesNotCleanupMergingSet(new ProcessingTimeAdaptor());
 	}
 
-	public void testOnTimePurgeDoesNotCleanupMergingSet(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testOnTimePurgeDoesNotCleanupMergingSet(TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		MergingWindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1663,7 +1664,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testNoGarbageCollectionTimerForGlobalWindow(new ProcessingTimeAdaptor());
 	}
 
-	public void testNoGarbageCollectionTimerForGlobalWindow(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testNoGarbageCollectionTimerForGlobalWindow(TimeDomainAdaptor timeAdaptor) throws Exception {
 
 		WindowAssigner<Integer, GlobalWindow> mockAssigner = mockGlobalWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
@@ -1767,7 +1768,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testGarbageCollectionTimer(new ProcessingTimeAdaptor());
 	}
 
-	public void testGarbageCollectionTimer(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testGarbageCollectionTimer(TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -1812,7 +1813,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testTriggerTimerAndGarbageCollectionTimerCoincide(new ProcessingTimeAdaptor());
 	}
 
-	public void testTriggerTimerAndGarbageCollectionTimerCoincide(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testTriggerTimerAndGarbageCollectionTimerCoincide(final TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -1868,7 +1869,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testStateAndTimerCleanupAtEventTimeGarbageCollection(new ProcessingTimeAdaptor());
 	}
 
-	public void testStateAndTimerCleanupAtEventTimeGarbageCollection(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testStateAndTimerCleanupAtEventTimeGarbageCollection(final TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -1938,7 +1939,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	 * Verify that we correctly clean up even when a purging trigger has purged
 	 * window state.
 	 */
-	public void testStateAndTimerCleanupAtEventTimeGCWithPurgingTrigger(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testStateAndTimerCleanupAtEventTimeGCWithPurgingTrigger(final TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -2009,7 +2010,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	 * Verify that we correctly clean up even when a purging trigger has purged
 	 * window state.
 	 */
-	public void testStateAndTimerCleanupAtGarbageCollectionWithPurgingTriggerAndMergingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testStateAndTimerCleanupAtGarbageCollectionWithPurgingTriggerAndMergingWindows(final TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -2075,7 +2076,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		testMergingWindowSetClearedAtGarbageCollection(new ProcessingTimeAdaptor());
 	}
 
-	public void testMergingWindowSetClearedAtGarbageCollection(TimeDomainAdaptor timeAdaptor) throws Exception {
+	private void testMergingWindowSetClearedAtGarbageCollection(TimeDomainAdaptor timeAdaptor) throws Exception {
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
@@ -2120,12 +2121,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		assertEquals(0, testHarness.getOutput().size());
 		assertEquals(0, testHarness.numKeyedStateEntries());
 
-		doAnswer(new Answer<TriggerResult>() {
-			@Override
-			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
-				return TriggerResult.FIRE;
-			}
-		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+		shouldFireOnElement(mockTrigger);
 
 		// 20 is just at the limit, window.maxTime() is 1 and allowed lateness is 20
 		testHarness.processWatermark(new Watermark(20));
@@ -2159,12 +2155,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		assertEquals(0, testHarness.getOutput().size());
 		assertEquals(0, testHarness.numKeyedStateEntries());
 
-		doAnswer(new Answer<TriggerResult>() {
-			@Override
-			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
-				return TriggerResult.FIRE;
-			}
-		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+		shouldFireOnElement(mockTrigger);
 
 		// window.maxTime() == 1 plus 20L of allowed lateness
 		testHarness.processWatermark(new Watermark(21));


[48/50] [abbrv] flink git commit: [FLINK-6165] [cep] Implement internal continuity for looping states.

Posted by fh...@apache.org.
[FLINK-6165] [cep] Implement internal continuity for looping states.

Allows looping states (oneOrMore, zeroOrMore, times) to specify
if they want their elements to be consecutive or allow non-matching
elements in-between.


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

Branch: refs/heads/table-retraction
Commit: aa3c395b97943e312dd16964b363f0e4f86c6739
Parents: d4665a0
Author: Dawid Wysakowicz <da...@getindata.com>
Authored: Mon Mar 27 15:05:11 2017 +0200
Committer: kl0u <kk...@gmail.com>
Committed: Thu Mar 30 10:24:19 2017 +0200

----------------------------------------------------------------------
 docs/dev/libs/cep.md                            |  66 ++++
 .../flink/cep/scala/pattern/Pattern.scala       |  29 ++
 .../java/org/apache/flink/cep/nfa/State.java    |   6 +-
 .../flink/cep/nfa/compiler/NFACompiler.java     | 189 +++++-----
 .../org/apache/flink/cep/pattern/Pattern.java   |  71 ++++
 .../apache/flink/cep/pattern/Quantifier.java    |  18 +-
 .../org/apache/flink/cep/nfa/NFAITCase.java     | 374 +++++++++++++++----
 7 files changed, 603 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/docs/dev/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index 932ba30..bb704c7 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -396,6 +396,7 @@ patternState.within(Time.seconds(10));
           <td>
               <p>Specifies that this pattern can occur zero or more times(kleene star). This means any number of events can be matched in this state.</p>
               <p>If eagerness is enabled(by default) for a pattern A*B and sequence A1 A2 B will generate patterns: B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_java">consecutive</a></p>
       {% highlight java %}
       patternState.zeroOrMore();
       {% endhighlight %}
@@ -406,6 +407,7 @@ patternState.within(Time.seconds(10));
           <td>
               <p>Specifies that this pattern can occur one or more times(kleene star). This means at least one and at most infinite number of events can be matched in this state.</p>
               <p>If eagerness is enabled (by default) for a pattern A*B and sequence A1 A2 B will generate patterns: A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_java">consecutive</a></p>
       {% highlight java %}
       patternState.oneOrMore();
       {% endhighlight %}
@@ -424,11 +426,50 @@ patternState.within(Time.seconds(10));
           <td><strong>Times</strong></td>
           <td>
               <p>Specifies exact number of times that this pattern should be matched.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_java">consecutive</a></p>
       {% highlight java %}
       patternState.times(2);
       {% endhighlight %}
           </td>
        </tr>
+       <tr>
+          <td><strong>Consecutive</strong><a name="consecutive_java"></a></td>
+          <td>
+              <p>Works in conjunction with zeroOrMore, oneOrMore or times. Specifies that any not matching element breaks the loop.</p>
+              
+              <p>If not applied a relaxed continuity (as in followedBy) is used.</p>
+
+          <p>E.g. a pattern like:</p>
+      {% highlight java %}
+      Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+           @Override
+           public boolean filter(Event value) throws Exception {
+               return value.getName().equals("c");
+           }
+      })
+      .followedBy("middle").where(new SimpleCondition<Event>() {
+           @Override
+           public boolean filter(Event value) throws Exception {
+               return value.getName().equals("a");
+           }
+      })
+      .oneOrMore(true).consecutive()
+      .followedBy("end1").where(new SimpleCondition<Event>() {
+           @Override
+           public boolean filter(Event value) throws Exception {
+               return value.getName().equals("b");
+           }
+      });
+      {% endhighlight %}
+
+             <p>Will generate the following matches for a sequence: C D A1 A2 A3 D A4 B</p>
+
+             <p>with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}</p>
+             <p>without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}</p>
+
+             <p><b>NOTICE:</b> This option can be applied only to zeroOrMore(), oneOrMore() and times()!</p>
+          </td>
+       </tr>
   </tbody>
 </table>
 </div>
@@ -511,6 +552,7 @@ patternState.within(Time.seconds(10))
           <td>
               <p>Specifies that this pattern can occur zero or more times(kleene star). This means any number of events can be matched in this state.</p>
               <p>If eagerness is enabled(by default) for a pattern A*B and sequence A1 A2 B will generate patterns: B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_scala">consecutive</a></p>
       {% highlight scala %}
       patternState.zeroOrMore()
       {% endhighlight %}
@@ -521,6 +563,7 @@ patternState.within(Time.seconds(10))
           <td>
               <p>Specifies that this pattern can occur one or more times(kleene star). This means at least one and at most infinite number of events can be matched in this state.</p>
               <p>If eagerness is enabled (by default) for a pattern A*B and sequence A1 A2 B will generate patterns: A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_scala">consecutive</a></p>
       {% highlight scala %}
       patternState.oneOrMore()
       {% endhighlight %}
@@ -539,11 +582,34 @@ patternState.within(Time.seconds(10))
           <td><strong>Times</strong></td>
           <td>
               <p>Specifies exact number of times that this pattern should be matched.</p>
+              <p>By default a relaxed internal continuity (between subsequent events of a loop) is used. For more info on the internal continuity see <a href="#consecutive_scala">consecutive</a></p>
       {% highlight scala %}
       patternState.times(2)
       {% endhighlight %}
           </td>
        </tr>
+       <tr>
+          <td><strong>Consecutive</strong><a name="consecutive_scala"></a></td>
+          <td>
+            <p>Works in conjunction with zeroOrMore, oneOrMore or times. Specifies that any not matching element breaks the loop.</p>
+            
+            <p>If not applied a relaxed continuity (as in followedBy) is used.</p>
+            
+      {% highlight scala %}
+      Pattern.begin("start").where(_.getName().equals("c"))
+       .followedBy("middle").where(_.getName().equals("a"))
+                            .oneOrMore(true).consecutive()
+       .followedBy("end1").where(_.getName().equals("b"));
+      {% endhighlight %}
+
+            <p>Will generate the following matches for a sequence: C D A1 A2 A3 D A4 B</p>
+
+            <p>with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}</p>
+            <p>without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}</p>
+
+            <p><b>NOTICE:</b> This option can be applied only to zeroOrMore(), oneOrMore() and times()!</p>
+          </td>
+       </tr>
   </tbody>
 </table>
 </div>

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
index 07dfc5a..c636029 100644
--- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
+++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
@@ -270,6 +270,35 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     this
   }
 
+
+  /**
+    * Works in conjunction with [[org.apache.flink.cep.scala.pattern.Pattern#zeroOrMore()]],
+    * [[org.apache.flink.cep.scala.pattern.Pattern#oneOrMore()]] or
+    * [[org.apache.flink.cep.scala.pattern.Pattern#times(int)]].
+    * Specifies that any not matching element breaks the loop.
+    *
+    * <p>E.g. a pattern like:
+    * {{{
+    * Pattern.begin("start").where(_.getName().equals("c"))
+    *        .followedBy("middle").where(_.getName().equals("a")).oneOrMore(true).consecutive()
+    *        .followedBy("end1").where(_.getName().equals("b"));
+    * }}}
+    *
+    * <p>for a sequence: C D A1 A2 A3 D A4 B
+    *
+    * <p>will generate matches: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}
+    *
+    * <p><b>NOTICE:</b> This operator can be applied only when either zeroOrMore,
+    * oneOrMore or times was previously applied!
+    *
+    * <p>By default a relaxed continuity is applied.
+    * @return pattern with continuity changed to strict
+    */
+  def consecutive(): Pattern[T, F] = {
+    jPattern.consecutive()
+    this
+  }
+
 }
 
 object Pattern {

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
index c673576..2503ffd 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
@@ -41,7 +41,7 @@ public class State<T> implements Serializable {
 	private static final long serialVersionUID = 6658700025989097781L;
 
 	private final String name;
-	private final StateType stateType;
+	private StateType stateType;
 	private final Collection<StateTransition<T>> stateTransitions;
 
 	public State(final String name, final StateType stateType) {
@@ -65,6 +65,10 @@ public class State<T> implements Serializable {
 		return stateTransitions;
 	}
 
+	public void makeStart() {
+		this.stateType = StateType.Start;
+	}
+
 	private void addStateTransition(
 			final StateTransitionAction action,
 			final State<T> targetState,

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 4fb918f..e441c4b 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -152,6 +152,7 @@ public class NFACompiler {
 
 		/**
 		 * Creates all the states between Start and Final state.
+		 *
 		 * @param sinkState the state that last state should point to (always the Final state)
 		 * @return the next state after Start in the resulting graph
 		 */
@@ -160,27 +161,25 @@ public class NFACompiler {
 			State<T> lastSink = sinkState;
 			while (currentPattern.getPrevious() != null) {
 				checkPatternNameUniqueness();
-
-				State<T> sourceState = new State<>(currentPattern.getName(), State.StateType.Normal);
-				states.add(sourceState);
-				usedNames.add(sourceState.getName());
+				usedNames.add(currentPattern.getName());
 
 				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.LOOPING)) {
-					convertToLooping(sourceState, lastSink);
+					final State<T> looping = createLooping(lastSink);
 
 					if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.AT_LEAST_ONE)) {
-						sourceState = createFirstMandatoryStateOfLoop(sourceState, State.StateType.Normal);
-						states.add(sourceState);
-						usedNames.add(sourceState.getName());
+						lastSink = createFirstMandatoryStateOfLoop(looping);
+					} else if (currentPattern instanceof FollowedByPattern &&
+								currentPattern.getQuantifier().hasProperty(QuantifierProperty.STRICT)) {
+						lastSink = createWaitingStateForZeroOrMore(looping, lastSink);
+					} else {
+						lastSink = looping;
 					}
-				} else if (currentPattern.getQuantifier() == Quantifier.TIMES) {
-					sourceState = convertToTimesState(sourceState, lastSink, currentPattern.getTimes());
+				} else if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.TIMES)) {
+					lastSink = createTimesState(lastSink, currentPattern.getTimes());
 				} else {
-					convertToSingletonState(sourceState, lastSink);
+					lastSink = createSingletonState(lastSink);
 				}
-
 				currentPattern = currentPattern.getPrevious();
-				lastSink = sourceState;
 
 				final Time currentWindowTime = currentPattern.getWindowTime();
 				if (currentWindowTime != null && currentWindowTime.toMilliseconds() < windowTime) {
@@ -192,6 +191,30 @@ public class NFACompiler {
 			return lastSink;
 		}
 
+		/**
+		 * Creates a pair of states that enables relaxed strictness before a zeroOrMore looping state.
+		 *
+		 * @param loopingState the first state of zeroOrMore complex state
+		 * @param lastSink     the state that the looping one points to
+		 * @return the newly created state
+		 */
+		private State<T> createWaitingStateForZeroOrMore(final State<T> loopingState, final State<T> lastSink) {
+			final State<T> followByState = createNormalState();
+			final State<T> followByStateWithoutProceed = createNormalState();
+
+			final IterativeCondition<T> currentFunction = (IterativeCondition<T>)currentPattern.getCondition();
+			final IterativeCondition<T> ignoreFunction = getIgnoreCondition(currentPattern);
+
+			followByState.addProceed(lastSink, BooleanConditions.<T>trueFunction());
+			followByState.addIgnore(followByStateWithoutProceed, ignoreFunction);
+			followByState.addTake(loopingState, currentFunction);
+
+			followByStateWithoutProceed.addIgnore(ignoreFunction);
+			followByStateWithoutProceed.addTake(loopingState, currentFunction);
+
+			return followByState;
+		}
+
 		private void checkPatternNameUniqueness() {
 			if (usedNames.contains(currentPattern.getName())) {
 				throw new MalformedPatternException(
@@ -202,110 +225,112 @@ public class NFACompiler {
 
 		/**
 		 * Creates the Start {@link State} of the resulting NFA graph.
+		 *
 		 * @param sinkState the state that Start state should point to (alwyas first state of middle states)
 		 * @return created state
 		 */
 		@SuppressWarnings("unchecked")
 		private State<T> createStartState(State<T> sinkState) {
 			checkPatternNameUniqueness();
+			usedNames.add(currentPattern.getName());
 
 			final State<T> beginningState;
 			if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.LOOPING)) {
-				final State<T> loopingState;
+				final State<T> loopingState = createLooping(sinkState);
 				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.AT_LEAST_ONE)) {
-					loopingState = new State<>(currentPattern.getName(), State.StateType.Normal);
-					beginningState = createFirstMandatoryStateOfLoop(loopingState, State.StateType.Start);
-					states.add(loopingState);
+					beginningState = createFirstMandatoryStateOfLoop(loopingState);
 				} else {
-					loopingState = new State<>(currentPattern.getName(), State.StateType.Start);
 					beginningState = loopingState;
 				}
-				convertToLooping(loopingState, sinkState, true);
-			} else  {
-				if (currentPattern.getQuantifier() == Quantifier.TIMES && currentPattern.getTimes() > 1) {
-					final State<T> timesState = new State<>(currentPattern.getName(), State.StateType.Normal);
-					states.add(timesState);
-					sinkState = convertToTimesState(timesState, sinkState, currentPattern.getTimes() - 1);
-				}
-
-				beginningState = new State<>(currentPattern.getName(), State.StateType.Start);
-				convertToSingletonState(beginningState, sinkState);
+			} else if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.TIMES)) {
+				beginningState = createTimesState(sinkState, currentPattern.getTimes());
+			} else {
+				beginningState = createSingletonState(sinkState);
 			}
 
-			states.add(beginningState);
-			usedNames.add(beginningState.getName());
+			beginningState.makeStart();
 
 			return beginningState;
 		}
 
 		/**
-		 * Converts the given state into a "complex" state consisting of given number of states with
+		 * Creates a "complex" state consisting of given number of states with
 		 * same {@link IterativeCondition}
 		 *
-		 * @param sourceState the state to be converted
-		 * @param sinkState the state that the converted state should point to
-		 * @param times number of times the state should be copied
+		 * @param sinkState the state that the created state should point to
+		 * @param times     number of times the state should be copied
 		 * @return the first state of the "complex" state, next state should point to it
 		 */
-		private State<T> convertToTimesState(final State<T> sourceState, final State<T> sinkState, int times) {
-			convertToSingletonState(sourceState, sinkState);
-			State<T> lastSink;
-			State<T> firstState = sourceState;
+		private State<T> createTimesState(final State<T> sinkState, int times) {
+			State<T> lastSink = sinkState;
 			for (int i = 0; i < times - 1; i++) {
-				lastSink = firstState;
-				firstState = new State<>(currentPattern.getName(), State.StateType.Normal);
-				states.add(firstState);
-				convertToSingletonState(firstState, lastSink);
+				lastSink = createSingletonState(
+					lastSink,
+					currentPattern instanceof FollowedByPattern &&
+					!currentPattern.getQuantifier().hasProperty(QuantifierProperty.STRICT));
 			}
-			return firstState;
+			return createSingletonState(lastSink, currentPattern instanceof FollowedByPattern);
 		}
 
 		/**
-		 * Converts the given state into a simple single state. For an OPTIONAL state it also consists
+		 * Creates a simple single state. For an OPTIONAL state it also consists
 		 * of a similar state without the PROCEED edge, so that for each PROCEED transition branches
 		 * in computation state graph  can be created only once.
 		 *
-		 * @param sourceState the state to be converted
 		 * @param sinkState state that the state being converted should point to
+		 * @return the created state
 		 */
 		@SuppressWarnings("unchecked")
-		private void convertToSingletonState(final State<T> sourceState, final State<T> sinkState) {
+		private State<T> createSingletonState(final State<T> sinkState) {
+			return createSingletonState(sinkState, currentPattern instanceof FollowedByPattern);
+		}
 
+		/**
+		 * Creates a simple single state. For an OPTIONAL state it also consists
+		 * of a similar state without the PROCEED edge, so that for each PROCEED transition branches
+		 * in computation state graph  can be created only once.
+		 *
+		 * @param addIgnore if any IGNORE should be added
+		 * @param sinkState state that the state being converted should point to
+		 * @return the created state
+		 */
+		@SuppressWarnings("unchecked")
+		private State<T> createSingletonState(final State<T> sinkState, boolean addIgnore) {
 			final IterativeCondition<T> currentFilterFunction = (IterativeCondition<T>) currentPattern.getCondition();
 			final IterativeCondition<T> trueFunction = BooleanConditions.trueFunction();
 
-			sourceState.addTake(sinkState, currentFilterFunction);
+			final State<T> singletonState = createNormalState();
+			singletonState.addTake(sinkState, currentFilterFunction);
 
 			if (currentPattern.getQuantifier() == Quantifier.OPTIONAL) {
-				sourceState.addProceed(sinkState, trueFunction);
+				singletonState.addProceed(sinkState, trueFunction);
 			}
 
-			if (currentPattern instanceof FollowedByPattern) {
+			if (addIgnore) {
 				final State<T> ignoreState;
 				if (currentPattern.getQuantifier() == Quantifier.OPTIONAL) {
-					ignoreState = new State<>(currentPattern.getName(), State.StateType.Normal);
+					ignoreState = createNormalState();
 					ignoreState.addTake(sinkState, currentFilterFunction);
-					states.add(ignoreState);
 				} else {
-					ignoreState = sourceState;
+					ignoreState = singletonState;
 				}
-				sourceState.addIgnore(ignoreState, trueFunction);
+				singletonState.addIgnore(ignoreState, trueFunction);
 			}
+			return singletonState;
 		}
 
 		/**
-		 * Patterns with quantifiers AT_LEAST_ONE_* are converted into pair of states: a singleton state and
+		 * Patterns with quantifiers AT_LEAST_ONE_* are created as a pair of states: a singleton state and
 		 * looping state. This method creates the first of the two.
 		 *
 		 * @param sinkState the state the newly created state should point to, it should be a looping state
-		 * @param stateType the type of the created state, as the NFA graph can also start wit AT_LEAST_ONE_*
 		 * @return the newly created state
 		 */
 		@SuppressWarnings("unchecked")
-		private State<T> createFirstMandatoryStateOfLoop(final State<T> sinkState, final State.StateType stateType) {
+		private State<T> createFirstMandatoryStateOfLoop(final State<T> sinkState) {
 
 			final IterativeCondition<T> currentFilterFunction = (IterativeCondition<T>) currentPattern.getCondition();
-			final State<T> firstState = new State<>(currentPattern.getName(), stateType);
+			final State<T> firstState = createNormalState();
 
 			firstState.addTake(sinkState, currentFilterFunction);
 			if (currentPattern instanceof FollowedByPattern) {
@@ -316,49 +341,45 @@ public class NFACompiler {
 		}
 
 		/**
-		 * Converts the given state into looping one. Looping state is one with TAKE edge to itself and
+		 * Creates the given state as a looping one. Looping state is one with TAKE edge to itself and
 		 * PROCEED edge to the sinkState. It also consists of a similar state without the PROCEED edge, so that
 		 * for each PROCEED transition branches in computation state graph  can be created only once.
 		 *
-		 * <p>If this looping state is first of a graph we should treat the {@link Pattern} as {@link FollowedByPattern}
-		 * to enable combinations.
-		 *
-		 * @param sourceState  the state to converted
-		 * @param sinkState    the state that the converted state should point to
-		 * @param isFirstState if the looping state is first of a graph
+		 * @param sinkState the state that the converted state should point to
+		 * @return the first state of the created complex state
 		 */
 		@SuppressWarnings("unchecked")
-		private void convertToLooping(final State<T> sourceState, final State<T> sinkState, boolean isFirstState) {
+		private State<T> createLooping(final State<T> sinkState) {
 
+			final State<T> loopingState = createNormalState();
 			final IterativeCondition<T> filterFunction = (IterativeCondition<T>) currentPattern.getCondition();
-			final IterativeCondition<T> trueFunction = BooleanConditions.<T>trueFunction();
+			final IterativeCondition<T> trueFunction = BooleanConditions.trueFunction();
 
-			sourceState.addProceed(sinkState, trueFunction);
-			sourceState.addTake(filterFunction);
-			if (currentPattern instanceof FollowedByPattern || isFirstState) {
-				final State<T> ignoreState = new State<>(
-					currentPattern.getName(),
-					State.StateType.Normal);
+			loopingState.addProceed(sinkState, trueFunction);
+			loopingState.addTake(filterFunction);
+			if (!currentPattern.getQuantifier().hasProperty(QuantifierProperty.STRICT)) {
+				final State<T> ignoreState = createNormalState();
 
 				final IterativeCondition<T> ignoreCondition = getIgnoreCondition(currentPattern);
 
-				sourceState.addIgnore(ignoreState, ignoreCondition);
-				ignoreState.addTake(sourceState, filterFunction);
-				ignoreState.addIgnore(ignoreState, ignoreCondition);
-				states.add(ignoreState);
+				ignoreState.addTake(loopingState, filterFunction);
+				ignoreState.addIgnore(ignoreCondition);
+				loopingState.addIgnore(ignoreState, ignoreCondition);
 			}
+
+			return loopingState;
 		}
 
 		/**
-		 * Converts the given state into looping one. Looping state is one with TAKE edge to itself and
-		 * PROCEED edge to the sinkState. It also consists of a similar state without the PROCEED edge, so that
-		 * for each PROCEED transition branches in computation state graph  can be created only once.
+		 * Creates a state with {@link State.StateType#Normal} and adds it to the collection of created states.
+		 * Should be used instead of instantiating with new operator.
 		 *
-		 * @param sourceState the state to converted
-		 * @param sinkState   the state that the converted state should point to
+		 * @return the created state
 		 */
-		private void convertToLooping(final State<T> sourceState, final State<T> sinkState) {
-			convertToLooping(sourceState, sinkState, false);
+		private State<T> createNormalState() {
+			final State<T> state = new State<>(currentPattern.getName(), State.StateType.Normal);
+			states.add(state);
+			return state;
 		}
 
 		/**
@@ -381,7 +402,7 @@ public class NFACompiler {
 	 * has at most one TAKE and one IGNORE and name of each state is unique. No PROCEED transition is allowed!
 	 *
 	 * @param oldStartState dummy start state of old graph
-	 * @param <T> type of events
+	 * @param <T>           type of events
 	 * @return map of new states, where key is the name of a state and value is the state itself
 	 */
 	@Internal

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
index cd51788..14c3e2d 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
@@ -267,6 +267,76 @@ public class Pattern<T, F extends T> {
 	}
 
 	/**
+	 * Works in conjunction with {@link Pattern#zeroOrMore()}, {@link Pattern#oneOrMore()} or {@link Pattern#times(int)}.
+	 * Specifies that any not matching element breaks the loop.
+	 *
+	 * <p>E.g. a pattern like:
+	 * <pre>{@code
+	 * Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+	 *      @Override
+	 *      public boolean filter(Event value) throws Exception {
+	 *          return value.getName().equals("c");
+	 *      }
+	 * })
+	 * .followedBy("middle").where(new FilterFunction<Event>() {
+	 *      @Override
+	 *      public boolean filter(Event value) throws Exception {
+	 *          return value.getName().equals("a");
+	 *      }
+	 * })
+	 * }<b>.oneOrMore(true).consecutive()</b>{@code
+	 * .followedBy("end1").where(new FilterFunction<Event>() {
+	 *      @Override
+	 *      public boolean filter(Event value) throws Exception {
+	 *          return value.getName().equals("b");
+	 *      }
+	 * });
+	 * }</pre>
+	 *
+	 * <p>for a sequence: C D A1 A2 A3 D A4 B
+	 *
+	 * <p>will generate matches: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}
+	 *
+	 * <p><b>NOTICE:</b> This operator can be applied only when either zeroOrMore,
+	 * oneOrMore or times was previously applied!
+	 *
+	 * <p>By default a relaxed continuity is applied.
+	 *
+	 * @return pattern with continuity changed to strict
+	 */
+	public Pattern<T, F> consecutive() {
+		switch (this.quantifier) {
+
+			case ZERO_OR_MORE_EAGER:
+				this.quantifier = Quantifier.ZERO_OR_MORE_EAGER_STRICT;
+				break;
+			case ZERO_OR_MORE_COMBINATIONS:
+				this.quantifier = Quantifier.ZERO_OR_MORE_COMBINATIONS_STRICT;
+				break;
+			case ONE_OR_MORE_EAGER:
+				this.quantifier = Quantifier.ONE_OR_MORE_EAGER_STRICT;
+				break;
+			case ONE_OR_MORE_COMBINATIONS:
+				this.quantifier = Quantifier.ONE_OR_MORE_COMBINATIONS_STRICT;
+				break;
+			case TIMES:
+				this.quantifier = Quantifier.TIMES_STRICT;
+				break;
+			case ZERO_OR_MORE_COMBINATIONS_STRICT:
+			case ONE_OR_MORE_EAGER_STRICT:
+			case ONE_OR_MORE_COMBINATIONS_STRICT:
+			case ZERO_OR_MORE_EAGER_STRICT:
+			case TIMES_STRICT:
+				throw new MalformedPatternException("Strict continuity already applied! consecutive() called twice.");
+			case ONE:
+			case OPTIONAL:
+				throw new MalformedPatternException("Strict continuity cannot be applied to " + this.quantifier);
+		}
+
+		return this;
+	}
+
+	/**
 	 * Specifies that this pattern can occur zero or once.
 	 *
 	 * @return The same pattern with applied Kleene ? operator
@@ -300,4 +370,5 @@ public class Pattern<T, F extends T> {
 			throw new MalformedPatternException("Already applied quantifier to this Pattern. Current quantifier is: " + this.quantifier);
 		}
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
index 7abe9bd..9789072 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
@@ -23,12 +23,24 @@ public enum Quantifier {
 	ONE,
 	ZERO_OR_MORE_EAGER(QuantifierProperty.LOOPING, QuantifierProperty.EAGER),
 	ZERO_OR_MORE_COMBINATIONS(QuantifierProperty.LOOPING),
+	ZERO_OR_MORE_EAGER_STRICT(QuantifierProperty.EAGER, QuantifierProperty.STRICT, QuantifierProperty.LOOPING),
+	ZERO_OR_MORE_COMBINATIONS_STRICT(QuantifierProperty.STRICT, QuantifierProperty.LOOPING),
 	ONE_OR_MORE_EAGER(
 		QuantifierProperty.LOOPING,
 		QuantifierProperty.EAGER,
 		QuantifierProperty.AT_LEAST_ONE),
+	ONE_OR_MORE_EAGER_STRICT(
+		QuantifierProperty.STRICT,
+		QuantifierProperty.LOOPING,
+		QuantifierProperty.EAGER,
+		QuantifierProperty.AT_LEAST_ONE),
 	ONE_OR_MORE_COMBINATIONS(QuantifierProperty.LOOPING, QuantifierProperty.AT_LEAST_ONE),
-	TIMES,
+	ONE_OR_MORE_COMBINATIONS_STRICT(
+		QuantifierProperty.STRICT,
+		QuantifierProperty.LOOPING,
+		QuantifierProperty.AT_LEAST_ONE),
+	TIMES(QuantifierProperty.TIMES),
+	TIMES_STRICT(QuantifierProperty.TIMES, QuantifierProperty.STRICT),
 	OPTIONAL;
 
 	private final EnumSet<QuantifierProperty> properties;
@@ -48,7 +60,9 @@ public enum Quantifier {
 	public enum QuantifierProperty {
 		LOOPING,
 		EAGER,
-		AT_LEAST_ONE
+		AT_LEAST_ONE,
+		STRICT,
+		TIMES
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/aa3c395b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 197767e..da5f413 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -591,8 +591,9 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(startEvent, 1));
 		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
-		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
-		inputEvents.add(new StreamRecord<>(end1, 6));
+		inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 6));
+		inputEvents.add(new StreamRecord<>(end1, 7));
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
@@ -642,7 +643,6 @@ public class NFAITCase extends TestLogger {
 		), resultingPatterns);
 	}
 
-
 	@Test
 	public void testBeginWithZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
@@ -1129,7 +1129,7 @@ public class NFAITCase extends TestLogger {
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).zeroOrMore(false).followedBy("end").where(new SimpleCondition<Event>() {
+		}).zeroOrMore().consecutive().followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 7056763917392056548L;
 
 			@Override
@@ -1281,7 +1281,6 @@ public class NFAITCase extends TestLogger {
 		), resultingPatterns);
 	}
 
-
 	@Test
 	public void testTimes() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
@@ -1659,9 +1658,293 @@ public class NFAITCase extends TestLogger {
 		), resultingPatterns);
 	}
 
-	/**
-	 * Clearing SharedBuffer
-	 */
+
+	///////////////////////////////         Consecutive           ////////////////////////////////////////
+
+	private static class ConsecutiveData {
+		static final Event startEvent = new Event(40, "c", 1.0);
+		static final Event middleEvent1 = new Event(41, "a", 2.0);
+		static final Event middleEvent2 = new Event(42, "a", 3.0);
+		static final Event middleEvent3 = new Event(43, "a", 4.0);
+		static final Event middleEvent4 = new Event(43, "a", 5.0);
+		static final Event end = new Event(44, "b", 5.0);
+
+		private ConsecutiveData() {
+		}
+	}
+
+	@Test
+	public void testStrictCombinationsOneOrMore() {
+		List<List<Event>> resultingPatterns = testStrictOneOrMore(false);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent4, ConsecutiveData.end)
+		));
+	}
+
+	@Test
+	public void testStrictEagerOneOrMore() {
+		List<List<Event>> resultingPatterns = testStrictOneOrMore(true);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.end)
+		));
+	}
+
+	private List<List<Event>> testStrictOneOrMore(boolean eager) {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
+		inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 2));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 6));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent4, 7));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.end, 8));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore(eager).consecutive()
+			.followedBy("end1").where(new SimpleCondition<Event>() {
+				private static final long serialVersionUID = 5726188262756267490L;
+
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("b");
+				}
+			});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		return feedNFA(inputEvents, nfa);
+	}
+
+	@Test
+	public void testStrictEagerZeroOrMore() {
+		List<List<Event>> resultingPatterns = testStrictZeroOrMore(true);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.end)
+		));
+	}
+
+	@Test
+	public void testStrictCombinationsZeroOrMore() {
+		List<List<Event>> resultingPatterns = testStrictZeroOrMore(false);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.end)
+		));
+	}
+
+	private List<List<Event>> testStrictZeroOrMore(boolean eager) {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
+		inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 2));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(new Event(50, "d", 6.0), 5));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 6));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.end, 7));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(eager).consecutive().followedBy("end1").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		return feedNFA(inputEvents, nfa);
+	}
+
+
+	@Test
+	public void testTimesStrict() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
+		inputEvents.add(new StreamRecord<>(new Event(23, "f", 1.0), 2));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(new Event(23, "f", 1.0), 4));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 5));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 6));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.end, 7));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).times(2).consecutive().followedBy("end1").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3, ConsecutiveData.end)
+		));
+	}
+
+	@Test
+	public void testTimesNonStrict() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
+		inputEvents.add(new StreamRecord<>(new Event(23, "f", 1.0), 2));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(new Event(23, "f", 1.0), 4));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 5));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 6));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.end, 7));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).times(2).followedBy("end1").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent2, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent1, ConsecutiveData.middleEvent3, ConsecutiveData.end),
+			Lists.newArrayList(ConsecutiveData.startEvent, ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3, ConsecutiveData.end)
+		));
+	}
+
+	@Test
+	public void testStartWithZeroOrMoreStrict() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore().consecutive();
+
+		testStartWithOneOrZeroOrMoreStrict(pattern);
+	}
+
+	@Test
+	public void testStartWithOneOrMoreStrict() {
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore().consecutive();
+
+		testStartWithOneOrZeroOrMoreStrict(pattern);
+	}
+
+	private void testStartWithOneOrZeroOrMoreStrict(Pattern<Event, ?> pattern) {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 4));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 5));
+		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 6));
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+
+		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
+			Lists.newArrayList(ConsecutiveData.middleEvent1),
+			Lists.newArrayList(ConsecutiveData.middleEvent2, ConsecutiveData.middleEvent3),
+			Lists.newArrayList(ConsecutiveData.middleEvent2),
+			Lists.newArrayList(ConsecutiveData.middleEvent3)
+		));
+	}
+
+	///////////////////////////////     Clearing SharedBuffer     ////////////////////////////////////////
 
 	@Test
 	public void testTimesClearingBuffer() {
@@ -1934,17 +2217,7 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		List<List<Event>> resultingPatterns = new ArrayList<>();
-
-		for (StreamRecord<Event> inputEvent : inputEvents) {
-			Collection<Map<String, Event>> patterns = nfa.process(
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
-
-			for (Map<String, Event> p: patterns) {
-				resultingPatterns.add(new ArrayList<>(p.values()));
-			}
-		}
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
 		return resultingPatterns;
 	}
@@ -2019,17 +2292,7 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		List<List<Event>> resultingPatterns = new ArrayList<>();
-
-		for (StreamRecord<Event> inputEvent : inputEvents) {
-			Collection<Map<String, Event>> patterns = nfa.process(
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
-
-			for (Map<String, Event> p: patterns) {
-				resultingPatterns.add(new ArrayList<>(p.values()));
-			}
-		}
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
 		return resultingPatterns;
 	}
@@ -2068,17 +2331,7 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		List<List<Event>> resultingPatterns = new ArrayList<>();
-
-		for (StreamRecord<Event> inputEvent : inputEvents) {
-			Collection<Map<String, Event>> patterns = nfa.process(
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
-
-			for (Map<String, Event> p: patterns) {
-				resultingPatterns.add(new ArrayList<>(p.values()));
-			}
-		}
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
 		compareMaps(resultingPatterns,
 				Lists.<List<Event>>newArrayList(
@@ -2145,17 +2398,7 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		List<List<Event>> resultingPatterns = new ArrayList<>();
-
-		for (StreamRecord<Event> inputEvent : inputEvents) {
-			Collection<Map<String, Event>> patterns = nfa.process(
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
-
-			for (Map<String, Event> p: patterns) {
-				resultingPatterns.add(new ArrayList<>(p.values()));
-			}
-		}
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
 		compareMaps(resultingPatterns,
 				Lists.<List<Event>>newArrayList(
@@ -2212,17 +2455,7 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		List<List<Event>> resultingPatterns = new ArrayList<>();
-
-		for (StreamRecord<Event> inputEvent : inputEvents) {
-			Collection<Map<String, Event>> patterns = nfa.process(
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
-
-			for (Map<String, Event> p: patterns) {
-				resultingPatterns.add(new ArrayList<>(p.values()));
-			}
-		}
+		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
 		compareMaps(resultingPatterns,
 				Lists.<List<Event>>newArrayList(
@@ -2237,6 +2470,21 @@ public class NFAITCase extends TestLogger {
 		);
 	}
 
+	private List<List<Event>> feedNFA(List<StreamRecord<Event>> inputEvents, NFA<Event> nfa) {
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+		return resultingPatterns;
+	}
+
 	private void compareMaps(List<List<Event>> actual, List<List<Event>> expected) {
 		Assert.assertEquals(expected.size(), actual.size());
 


[02/50] [abbrv] flink git commit: [FLINK-5067][hotfix] Fix snapshot deployment

Posted by fh...@apache.org.
[FLINK-5067][hotfix] Fix snapshot deployment


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

Branch: refs/heads/table-retraction
Commit: 64c7b11984539db3ca065421071492ca23a2bb04
Parents: 36cf94a
Author: Robert Metzger <rm...@apache.org>
Authored: Wed Mar 22 17:01:41 2017 +0100
Committer: Robert Metzger <rm...@apache.org>
Committed: Wed Mar 22 17:01:41 2017 +0100

----------------------------------------------------------------------
 tools/deploy_to_maven.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/64c7b119/tools/deploy_to_maven.sh
----------------------------------------------------------------------
diff --git a/tools/deploy_to_maven.sh b/tools/deploy_to_maven.sh
index c818a42..16d529e 100755
--- a/tools/deploy_to_maven.sh
+++ b/tools/deploy_to_maven.sh
@@ -80,11 +80,11 @@ echo "detected current version as: '$CURRENT_FLINK_VERSION'"
 
 #
 # This script deploys our project to sonatype SNAPSHOTS.
-# It will deploy both a hadoop v1 and a hadoop v2 (yarn) artifact
+# It will deploy a hadoop v2 (yarn) artifact
 #
 
 if [[ $CURRENT_FLINK_VERSION == *SNAPSHOT* ]] ; then
-    MVN_SNAPSHOT_OPTS="-B -Pdocs-and-source -DskipTests -Drat.skip=true -Drat.ignoreErrors=true \
+    MVN_SNAPSHOT_OPTS="-B -Pdocs-and-source,jdk8 -DskipTests -Drat.skip=true -Drat.ignoreErrors=true \
         -DretryFailedDeploymentCount=10 --settings deploysettings.xml clean deploy"
 
     # hadoop2 scala 2.10


[06/50] [abbrv] flink git commit: [FLINK-3318] Add support for quantifiers to CEP's pattern API

Posted by fh...@apache.org.
[FLINK-3318] Add support for quantifiers to CEP's pattern API


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

Branch: refs/heads/table-retraction
Commit: 9001c4ef82a7a04d821252ac62bb7809a931c98a
Parents: d0695c0
Author: Dawid Wysakowicz <da...@getindata.com>
Authored: Sat Mar 18 20:53:00 2017 +0100
Committer: kl0u <kk...@gmail.com>
Committed: Thu Mar 23 10:47:55 2017 +0100

----------------------------------------------------------------------
 docs/dev/libs/cep.md                            |   78 +-
 .../flink/cep/scala/pattern/Pattern.scala       |   81 +-
 .../apache/flink/cep/nfa/ComputationState.java  |   44 +-
 .../org/apache/flink/cep/nfa/DeweyNumber.java   |   17 +-
 .../main/java/org/apache/flink/cep/nfa/NFA.java |  398 ++++--
 .../org/apache/flink/cep/nfa/SharedBuffer.java  |   25 +-
 .../java/org/apache/flink/cep/nfa/State.java    |   32 +-
 .../apache/flink/cep/nfa/StateTransition.java   |   20 +-
 .../flink/cep/nfa/StateTransitionAction.java    |    4 +-
 .../nfa/compiler/MalformedPatternException.java |   32 -
 .../flink/cep/nfa/compiler/NFACompiler.java     |  317 ++++-
 .../flink/cep/pattern/FilterFunctions.java      |   44 +
 .../cep/pattern/MalformedPatternException.java  |   32 +
 .../flink/cep/pattern/NotFilterFunction.java    |   42 +
 .../org/apache/flink/cep/pattern/Pattern.java   |  115 ++
 .../apache/flink/cep/pattern/Quantifier.java    |   54 +
 .../org/apache/flink/cep/nfa/NFAITCase.java     | 1338 +++++++++++++++++-
 .../java/org/apache/flink/cep/nfa/NFATest.java  |   90 +-
 .../flink/cep/nfa/compiler/NFACompilerTest.java |  152 +-
 .../apache/flink/cep/pattern/PatternTest.java   |   54 +
 20 files changed, 2595 insertions(+), 374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/docs/dev/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index 8047481..22cffbc 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -341,7 +341,45 @@ patternState.subtype(SubEvent.class);
 patternState.within(Time.seconds(10));
 {% endhighlight %}
           </td>
-      </tr>
+       </tr>
+       <tr>
+          <td><strong>ZeroOrMore</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur zero or more times(kleene star). This means any number of events can be matched in this state.</p>
+              <p>If eagerness is enabled(by default) for a pattern A*B and sequence A1 A2 B will generate patterns: B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.</p>
+      {% highlight java %}
+      patternState.zeroOrMore();
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>OneOrMore</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur one or more times(kleene star). This means at least one and at most infinite number of events can be matched in this state.</p>
+              <p>If eagerness is enabled (by default) for a pattern A*B and sequence A1 A2 B will generate patterns: A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.</p>
+      {% highlight java %}
+      patternState.oneOrMore();
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>Optional</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur zero or once.</p>
+      {% highlight java %}
+      patternState.optional();
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>Times</strong></td>
+          <td>
+              <p>Specifies exact number of times that this pattern should be matched.</p>
+      {% highlight java %}
+      patternState.times(2);
+      {% endhighlight %}
+          </td>
+       </tr>
   </tbody>
 </table>
 </div>
@@ -419,6 +457,44 @@ patternState.within(Time.seconds(10))
 {% endhighlight %}
           </td>
       </tr>
+       <tr>
+          <td><strong>ZeroOrMore</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur zero or more times(kleene star). This means any number of events can be matched in this state.</p>
+              <p>If eagerness is enabled(by default) for a pattern A*B and sequence A1 A2 B will generate patterns: B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.</p>
+      {% highlight scala %}
+      patternState.zeroOrMore()
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>OneOrMore</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur one or more times(kleene star). This means at least one and at most infinite number of events can be matched in this state.</p>
+              <p>If eagerness is enabled (by default) for a pattern A*B and sequence A1 A2 B will generate patterns: A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.</p>
+      {% highlight scala %}
+      patternState.oneOrMore()
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>Optional</strong></td>
+          <td>
+              <p>Specifies that this pattern can occur zero or once.</p>
+      {% highlight scala %}
+      patternState.optional()
+      {% endhighlight %}
+          </td>
+       </tr>
+       <tr>
+          <td><strong>Times</strong></td>
+          <td>
+              <p>Specifies exact number of times that this pattern should be matched.</p>
+      {% highlight scala %}
+      patternState.times(2)
+      {% endhighlight %}
+          </td>
+       </tr>
   </tbody>
 </table>
 </div>

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
index cc3b03c..5baf780 100644
--- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
+++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
@@ -19,7 +19,7 @@ package org.apache.flink.cep.scala.pattern
 
 import org.apache.flink.api.common.functions.FilterFunction
 import org.apache.flink.cep
-import org.apache.flink.cep.pattern.{Pattern => JPattern}
+import org.apache.flink.cep.pattern.{Quantifier, Pattern => JPattern}
 import org.apache.flink.streaming.api.windowing.time.Time
 
 /**
@@ -59,6 +59,12 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
 
   /**
     *
+    * @return currently applied quantifier to this pattern
+    */
+  def getQuantifier: Quantifier = jPattern.getQuantifier
+
+  /**
+    *
     * @return Filter condition for an event to be matched
     */
   def getFilterFunction(): Option[FilterFunction[F]] = {
@@ -160,6 +166,79 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     wrapPattern(jPattern.getPrevious())
   }
 
+  /**
+    * Specifies that this pattern can occur zero or more times(kleene star).
+    * This means any number of events can be matched in this state.
+    *
+    * @return The same pattern with applied Kleene star operator
+    */
+  def zeroOrMore: Pattern[T, F] = {
+    jPattern.zeroOrMore()
+    this
+  }
+
+  /**
+    * Specifies that this pattern can occur zero or more times(kleene star).
+    * This means any number of events can be matched in this state.
+    *
+    * If eagerness is enabled for a pattern A*B and sequence A1 A2 B will generate patterns:
+    * B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.
+    *
+    * @param eager if true the pattern always consumes earlier events
+    * @return The same pattern with applied Kleene star operator
+    */
+  def zeroOrMore(eager: Boolean): Pattern[T, F] = {
+    jPattern.zeroOrMore(eager)
+    this
+  }
+
+  /**
+    * Specifies that this pattern can occur one or more times(kleene star).
+    * This means at least one and at most infinite number of events can be matched in this state.
+    *
+    * @return The same pattern with applied Kleene plus operator
+    */
+  def oneOrMore: Pattern[T, F] = {
+    jPattern.oneOrMore()
+    this
+  }
+
+  /**
+    * Specifies that this pattern can occur one or more times(kleene star).
+    * This means at least one and at most infinite number of events can be matched in this state.
+    *
+    * If eagerness is enabled for a pattern A+B and sequence A1 A2 B will generate patterns:
+    * A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.
+    *
+    * @param eager if true the pattern always consumes earlier events
+    * @return The same pattern with applied Kleene plus operator
+    */
+  def oneOrMore(eager: Boolean): Pattern[T, F] = {
+    jPattern.oneOrMore(eager)
+    this
+  }
+
+  /**
+    * Specifies that this pattern can occur zero or once.
+    *
+    * @return The same pattern with applied Kleene ? operator
+    */
+  def optional: Pattern[T, F] = {
+    jPattern.optional()
+    this
+  }
+
+  /**
+    * Specifies exact number of times that this pattern should be matched.
+    *
+    * @param times number of times matching event must appear
+    * @return The same pattern with number of times applied
+    */
+  def times(times: Int): Pattern[T, F] = {
+    jPattern.times(times)
+    this
+  }
+
 }
 
 object Pattern {

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
index 3f44fba..445d038 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.cep.nfa;
 
+import org.apache.flink.util.Preconditions;
+
 /**
  * Helper class which encapsulates the state of the NFA computation. It points to the current state,
  * the last taken event, its occurrence timestamp, the current version and the starting timestamp
@@ -41,17 +43,21 @@ public class ComputationState<T> {
 	// Timestamp of the first element in the pattern
 	private final long startTimestamp;
 
-	public ComputationState(
-		final State<T> currentState,
-		final T event,
-		final long timestamp,
-		final DeweyNumber version,
-		final long startTimestamp) {
+	private final State<T> previousState;
+
+	private ComputationState(
+			final State<T> currentState,
+			final State<T> previousState,
+			final T event,
+			final long timestamp,
+			final DeweyNumber version,
+			final long startTimestamp) {
 		this.state = currentState;
 		this.event = event;
 		this.timestamp = timestamp;
 		this.version = version;
 		this.startTimestamp = startTimestamp;
+		this.previousState = previousState;
 	}
 
 	public boolean isFinalState() {
@@ -59,7 +65,7 @@ public class ComputationState<T> {
 	}
 
 	public boolean isStartState() {
-		return state.isStart();
+		return state.isStart() && event == null;
 	}
 
 	public long getTimestamp() {
@@ -74,6 +80,10 @@ public class ComputationState<T> {
 		return state;
 	}
 
+	public State<T> getPreviousState() {
+		return previousState;
+	}
+
 	public T getEvent() {
 		return event;
 	}
@@ -81,4 +91,24 @@ public class ComputationState<T> {
 	public DeweyNumber getVersion() {
 		return version;
 	}
+
+	public static <T> ComputationState<T> createStartState(final State<T> state) {
+		Preconditions.checkArgument(state.isStart());
+		return new ComputationState<>(state, null, null, -1L, new DeweyNumber(1), -1L);
+	}
+
+	public static <T> ComputationState<T> createStartState(final State<T> state, final DeweyNumber version) {
+		Preconditions.checkArgument(state.isStart());
+		return new ComputationState<>(state, null, null, -1L, version, -1L);
+	}
+
+	public static <T> ComputationState<T> createState(
+			final State<T> currentState,
+			final State<T> previousState,
+			final T event,
+			final long timestamp,
+			final DeweyNumber version,
+			final long startTimestamp) {
+		return new ComputationState<>(currentState, previousState, event, timestamp, version, startTimestamp);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
index bb9039d..fd3fafa 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
@@ -44,6 +44,10 @@ public class DeweyNumber implements Serializable {
 		this.deweyNumber = deweyNumber;
 	}
 
+	public DeweyNumber(DeweyNumber number) {
+		this.deweyNumber = Arrays.copyOf(number.deweyNumber, number.deweyNumber.length);
+	}
+
 	/**
 	 * Checks whether this dewey number is compatible to the other dewey number.
 	 *
@@ -90,8 +94,19 @@ public class DeweyNumber implements Serializable {
 	 * @return A new dewey number derived from this whose last digit is increased by one
 	 */
 	public DeweyNumber increase() {
+		return increase(1);
+	}
+
+	/**
+	 * Creates a new dewey number from this such that its last digit is increased by the supplied
+	 * number
+	 *
+	 * @param times how many times to increase the Dewey number
+	 * @return A new dewey number derived from this whose last digit is increased by given number
+	 */
+	public DeweyNumber increase(int times) {
 		int[] newDeweyNumber = Arrays.copyOf(deweyNumber, deweyNumber.length);
-		newDeweyNumber[deweyNumber.length - 1]++;
+		newDeweyNumber[deweyNumber.length - 1] += times;
 
 		return new DeweyNumber(newDeweyNumber);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 257418a..3d42248 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -19,6 +19,7 @@
 package org.apache.flink.cep.nfa;
 
 import com.google.common.collect.LinkedHashMultimap;
+import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
@@ -39,7 +40,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -87,7 +87,7 @@ public class NFA<T> implements Serializable {
 	/**
 	 * 	Buffer used to store the matched events.
 	 */
-	private final SharedBuffer<State<T>, T> sharedBuffer;
+	private final SharedBuffer<String, T> sharedBuffer;
 
 	/**
 	 * A set of all the valid NFA states, as returned by the
@@ -98,7 +98,7 @@ public class NFA<T> implements Serializable {
 
 	/**
 	 * The length of a windowed pattern, as specified using the
-	 * {@link org.apache.flink.cep.pattern.Pattern#within(Time) Pattern.within(Time)}
+	 * {@link org.apache.flink.cep.pattern.Pattern#within(Time)}  Pattern.within(Time)}
 	 * method.
 	 */
 	private final long windowTime;
@@ -109,9 +109,6 @@ public class NFA<T> implements Serializable {
 	 */
 	private final boolean handleTimeout;
 
-	// Current starting index for the next dewey version number
-	private int startEventCounter;
-
 	/**
 	 * Current set of {@link ComputationState computation states} within the state machine.
 	 * These are the "active" intermediate states that are waiting for new matching
@@ -119,8 +116,6 @@ public class NFA<T> implements Serializable {
 	 */
 	private transient Queue<ComputationState<T>> computationStates;
 
-	private StateTransitionComparator<T>  stateTransitionComparator;
-
 	public NFA(
 			final TypeSerializer<T> eventSerializer,
 			final long windowTime,
@@ -129,11 +124,10 @@ public class NFA<T> implements Serializable {
 		this.nonDuplicatingTypeSerializer = new NonDuplicatingTypeSerializer<>(eventSerializer);
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
-		this.sharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
-		this.computationStates = new LinkedList<>();
-		this.states = new HashSet<>();
-		this.startEventCounter = 1;
-		this.stateTransitionComparator =  new StateTransitionComparator<>();
+		sharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
+		computationStates = new LinkedList<>();
+
+		states = new HashSet<>();
 	}
 
 	public Set<State<T>> getStates() {
@@ -150,7 +144,7 @@ public class NFA<T> implements Serializable {
 		states.add(state);
 
 		if (state.isStart()) {
-			computationStates.add(new ComputationState<>(state, null, -1L, null, -1L));
+			computationStates.add(ComputationState.createStartState(state));
 		}
 	}
 
@@ -201,8 +195,8 @@ public class NFA<T> implements Serializable {
 				}
 
 				// remove computation state which has exceeded the window length
-				sharedBuffer.release(computationState.getState(), computationState.getEvent(), computationState.getTimestamp());
-				sharedBuffer.remove(computationState.getState(), computationState.getEvent(), computationState.getTimestamp());
+				sharedBuffer.release(computationState.getState().getName(), computationState.getEvent(), computationState.getTimestamp());
+				sharedBuffer.remove(computationState.getState().getName(), computationState.getEvent(), computationState.getTimestamp());
 
 				newComputationStates = Collections.emptyList();
 			} else if (event != null) {
@@ -218,8 +212,8 @@ public class NFA<T> implements Serializable {
 					result.addAll(matches);
 
 					// remove found patterns because they are no longer needed
-					sharedBuffer.release(newComputationState.getState(), newComputationState.getEvent(), newComputationState.getTimestamp());
-					sharedBuffer.remove(newComputationState.getState(), newComputationState.getEvent(), newComputationState.getTimestamp());
+					sharedBuffer.release(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
+					sharedBuffer.remove(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
 				} else {
 					// add new computation state; it will be processed once the next event arrives
 					computationStates.add(newComputationState);
@@ -252,8 +246,7 @@ public class NFA<T> implements Serializable {
 			return nonDuplicatingTypeSerializer.equals(other.nonDuplicatingTypeSerializer) &&
 				sharedBuffer.equals(other.sharedBuffer) &&
 				states.equals(other.states) &&
-				windowTime == other.windowTime &&
-				startEventCounter == other.startEventCounter;
+				windowTime == other.windowTime;
 		} else {
 			return false;
 		}
@@ -261,12 +254,80 @@ public class NFA<T> implements Serializable {
 
 	@Override
 	public int hashCode() {
-		return Objects.hash(nonDuplicatingTypeSerializer, sharedBuffer, states, windowTime, startEventCounter);
+		return Objects.hash(nonDuplicatingTypeSerializer, sharedBuffer, states, windowTime);
+	}
+
+	private static <T> boolean isEquivalentState(final State<T> s1, final State<T> s2) {
+		return s1.getName().equals(s2.getName());
 	}
 
 	/**
+	 * Class for storing resolved transitions. It counts at insert time the number of
+	 * branching transitions both for IGNORE and TAKE actions.
+ 	 */
+	private static class OutgoingEdges<T> {
+		private List<StateTransition<T>> edges = new ArrayList<>();
+
+		private final State<T> currentState;
+
+		private int totalTakeBranches = 0;
+		private int totalIgnoreBranches = 0;
+
+		OutgoingEdges(final State<T> currentState) {
+			this.currentState = currentState;
+		}
+
+		void add(StateTransition<T> edge) {
+
+			if (!isSelfIgnore(edge)) {
+				if (edge.getAction() == StateTransitionAction.IGNORE) {
+					totalIgnoreBranches++;
+				} else if (edge.getAction() == StateTransitionAction.TAKE) {
+					totalTakeBranches++;
+				}
+			}
+
+			edges.add(edge);
+		}
+
+		int getTotalIgnoreBranches() {
+			return totalIgnoreBranches;
+		}
+		int getTotalTakeBranches() {
+			return totalTakeBranches;
+		}
+
+		List<StateTransition<T>> getEdges() {
+			return edges;
+		}
+
+		private boolean isSelfIgnore(final StateTransition<T> edge) {
+			return isEquivalentState(edge.getTargetState(), currentState) &&
+				edge.getAction() == StateTransitionAction.IGNORE;
+		}
+	}
+
+
+	/**
 	 * Computes the next computation states based on the given computation state, the current event,
-	 * its timestamp and the internal state machine.
+	 * its timestamp and the internal state machine. The algorithm is:
+	 *
+	 * 1. Decide on valid transitions and number of branching paths. See {@link OutgoingEdges}
+	 * 2. Perform transitions:
+	 *      a) IGNORE (links in {@link SharedBuffer} will still point to the previous event)
+	 *          - do not perform for Start State - special case
+	 *          - if stays in the same state increase the current stage for future use with number of
+	 *            outgoing edges
+	 *          - if after PROCEED increase current stage and add new stage (as we change the state)
+	 *          - lock the entry in {@link SharedBuffer} as it is needed in the created branch
+	 *      b) TAKE (links in {@link SharedBuffer} will point to the current event)
+	 *          - add entry to the shared buffer with version of the current computation state
+	 *          - add stage and then increase with number of takes for the future computation states
+	 *          - peek to the next state if it has PROCEED path to a Final State, if true create
+	 *            Final ComputationState to emit results
+	 * 3. Handle the Start State, as it always have to remain
+	 * 4. Release the corresponding entries in {@link SharedBuffer}.
+	 *
 	 *
 	 * @param computationState Current computation state
 	 * @param event Current event which is processed
@@ -277,31 +338,179 @@ public class NFA<T> implements Serializable {
 			final ComputationState<T> computationState,
 			final T event,
 			final long timestamp) {
-		Stack<State<T>> states = new Stack<>();
-		List<ComputationState<T>> resultingComputationStates = new ArrayList<>();
-		State<T> state = computationState.getState();
 
-		states.push(state);
+		final OutgoingEdges<T> outgoingEdges = createDecisionGraph(computationState, event);
+
+		// Create the computing version based on the previously computed edges
+		// We need to defer the creation of computation states until we know how many edges start
+		// at this computation state so that we can assign proper version
+		final List<StateTransition<T>> edges = outgoingEdges.getEdges();
+		int takeBranchesToVisit = Math.max(0, outgoingEdges.getTotalTakeBranches() - 1);
+		int ignoreBranchesToVisit = outgoingEdges.getTotalIgnoreBranches();
+
+		final List<ComputationState<T>> resultingComputationStates = new ArrayList<>();
+		for (StateTransition<T> edge : edges) {
+			switch (edge.getAction()) {
+				case IGNORE: {
+					if (!computationState.isStartState()) {
+						final DeweyNumber version;
+						if (isEquivalentState(edge.getTargetState(), computationState.getState())) {
+							//Stay in the same state (it can be either looping one or singleton)
+							final int toIncrease = calculateIncreasingSelfState(
+								outgoingEdges.getTotalIgnoreBranches(),
+								outgoingEdges.getTotalTakeBranches());
+							version = computationState.getVersion().increase(toIncrease);
+						} else {
+							//IGNORE after PROCEED
+							version = computationState.getVersion().increase(ignoreBranchesToVisit).addStage();
+							ignoreBranchesToVisit--;
+						}
 
-		boolean branched = false;
-		while (!states.isEmpty()) {
-			State<T> currentState = states.pop();
-			final List<StateTransition<T>> stateTransitions = new ArrayList<>(currentState.getStateTransitions());
+						resultingComputationStates.add(
+							ComputationState.createState(
+								edge.getTargetState(),
+								computationState.getPreviousState(),
+								computationState.getEvent(),
+								computationState.getTimestamp(),
+								version,
+								computationState.getStartTimestamp()
+							)
+						);
+						sharedBuffer.lock(
+							computationState.getPreviousState().getName(),
+							computationState.getEvent(),
+							computationState.getTimestamp());
+					}
+				}
+				break;
+				case TAKE:
+					final State<T> newState = edge.getTargetState();
+					final State<T> consumingState = edge.getSourceState();
+					final State<T> previousEventState = computationState.getPreviousState();
+
+					final T previousEvent = computationState.getEvent();
+					final DeweyNumber currentVersion = computationState.getVersion();
+
+					final DeweyNumber newComputationStateVersion = new DeweyNumber(currentVersion).addStage().increase(takeBranchesToVisit);
+					takeBranchesToVisit--;
+
+					final long startTimestamp;
+					if (computationState.isStartState()) {
+						startTimestamp = timestamp;
+						sharedBuffer.put(
+							consumingState.getName(),
+							event,
+							timestamp,
+							currentVersion);
+					} else {
+						startTimestamp = computationState.getStartTimestamp();
+						sharedBuffer.put(
+							consumingState.getName(),
+							event,
+							timestamp,
+							previousEventState.getName(),
+							previousEvent,
+							computationState.getTimestamp(),
+							currentVersion);
+					}
 
-			// this is for when we restore from legacy. In that case, the comparator is null
-			// as it did not exist in the previous Flink versions, so we have to initialize it here.
+					// a new computation state is referring to the shared entry
+					sharedBuffer.lock(consumingState.getName(), event, timestamp);
+
+					resultingComputationStates.add(ComputationState.createState(
+						newState,
+						consumingState,
+						event,
+						timestamp,
+						newComputationStateVersion,
+						startTimestamp
+					));
+
+					//check if newly created state is optional (have a PROCEED path to Final state)
+					final State<T> finalState = findFinalStateAfterProceed(newState, event);
+					if (finalState != null) {
+						sharedBuffer.lock(consumingState.getName(), event, timestamp);
+						resultingComputationStates.add(ComputationState.createState(
+							finalState,
+							consumingState,
+							event,
+							timestamp,
+							newComputationStateVersion,
+							startTimestamp));
+					}
+					break;
+			}
+		}
 
-			if (stateTransitionComparator == null) {
-				stateTransitionComparator = new StateTransitionComparator();
+		if (computationState.isStartState()) {
+			final int totalBranches = calculateIncreasingSelfState(outgoingEdges.getTotalIgnoreBranches(), outgoingEdges.getTotalTakeBranches());
+			final ComputationState<T> startState = createStartState(computationState, totalBranches);
+			resultingComputationStates.add(startState);
+		}
+
+		if (computationState.getEvent() != null) {
+			// release the shared entry referenced by the current computation state.
+			sharedBuffer.release(
+				computationState.getPreviousState().getName(),
+				computationState.getEvent(),
+				computationState.getTimestamp());
+			// try to remove unnecessary shared buffer entries
+			sharedBuffer.remove(
+				computationState.getPreviousState().getName(),
+				computationState.getEvent(),
+				computationState.getTimestamp());
+		}
+
+		return resultingComputationStates;
+	}
+
+	private State<T> findFinalStateAfterProceed(State<T> state, T event) {
+		final Stack<State<T>> statesToCheck = new Stack<>();
+		statesToCheck.push(state);
+
+		try {
+			while (!statesToCheck.isEmpty()) {
+				final State<T> currentState = statesToCheck.pop();
+				for (StateTransition<T> transition : currentState.getStateTransitions()) {
+					if (transition.getAction() == StateTransitionAction.PROCEED &&
+						checkFilterCondition(transition.getCondition(), event)) {
+						if (transition.getTargetState().isFinal()) {
+							return transition.getTargetState();
+						} else {
+							statesToCheck.push(transition.getTargetState());
+						}
+					}
+				}
 			}
+		} catch (Exception e) {
+			throw new RuntimeException("Failure happened in filter function.", e);
+		}
+
+		return null;
+	}
+
+	private int calculateIncreasingSelfState(int ignoreBranches, int takeBranches) {
+		return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + 1;
+	}
+
+	private ComputationState<T> createStartState(final ComputationState<T> computationState, final int totalBranches) {
+		final DeweyNumber startVersion = computationState.getVersion().increase(totalBranches);
+		return ComputationState.createStartState(computationState.getState(), startVersion);
+	}
 
-			// impose the IGNORE will be processed last
-			Collections.sort(stateTransitions, stateTransitionComparator);
+	private OutgoingEdges<T> createDecisionGraph(ComputationState<T> computationState, T event) {
+		final Stack<State<T>> states = new Stack<>();
+		states.push(computationState.getState());
+		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(computationState.getState());
+		//First create all outgoing edges, so to be able to reason about the Dewey version
+		while (!states.isEmpty()) {
+			State<T> currentState = states.pop();
+			Collection<StateTransition<T>> stateTransitions = currentState.getStateTransitions();
 
 			// check all state transitions for each state
-			for (StateTransition<T> stateTransition: stateTransitions) {
+			for (StateTransition<T> stateTransition : stateTransitions) {
 				try {
-					if (stateTransition.getCondition() == null || stateTransition.getCondition().filter(event)) {
+					if (checkFilterCondition(stateTransition.getCondition(), event)) {
 						// filter condition is true
 						switch (stateTransition.getAction()) {
 							case PROCEED:
@@ -310,73 +519,8 @@ public class NFA<T> implements Serializable {
 								states.push(stateTransition.getTargetState());
 								break;
 							case IGNORE:
-								final DeweyNumber version;
-								if (branched) {
-									version = computationState.getVersion().increase();
-								} else {
-									version = computationState.getVersion();
-								}
-								resultingComputationStates.add(new ComputationState<T>(
-									computationState.getState(),
-									computationState.getEvent(),
-									computationState.getTimestamp(),
-									version,
-									computationState.getStartTimestamp()));
-
-								// we have a new computation state referring to the same the shared entry
-								// the lock of the current computation is released later on
-								sharedBuffer.lock(computationState.getState(), computationState.getEvent(), computationState.getTimestamp());
-								break;
 							case TAKE:
-								final State<T> newState = stateTransition.getTargetState();
-								final DeweyNumber oldVersion;
-								final DeweyNumber newComputationStateVersion;
-								final State<T> previousState = computationState.getState();
-								final T previousEvent = computationState.getEvent();
-								final long previousTimestamp;
-								final long startTimestamp;
-
-								if (computationState.isStartState()) {
-									oldVersion = new DeweyNumber(startEventCounter++);
-									newComputationStateVersion = oldVersion.addStage();
-									startTimestamp = timestamp;
-									previousTimestamp = -1L;
-
-								} else {
-									startTimestamp = computationState.getStartTimestamp();
-									previousTimestamp = computationState.getTimestamp();
-									oldVersion = computationState.getVersion();
-
-									branched = true;
-									newComputationStateVersion = oldVersion.addStage();
-								}
-
-								if (previousState.isStart()) {
-									sharedBuffer.put(
-										newState,
-										event,
-										timestamp,
-										oldVersion);
-								} else {
-									sharedBuffer.put(
-										newState,
-										event,
-										timestamp,
-										previousState,
-										previousEvent,
-										previousTimestamp,
-										oldVersion);
-								}
-
-								// a new computation state is referring to the shared entry
-								sharedBuffer.lock(newState, event, timestamp);
-
-								resultingComputationStates.add(new ComputationState<T>(
-									newState,
-									event,
-									timestamp,
-									newComputationStateVersion,
-									startTimestamp));
+								outgoingEdges.add(stateTransition);
 								break;
 						}
 					}
@@ -385,19 +529,12 @@ public class NFA<T> implements Serializable {
 				}
 			}
 		}
+		return outgoingEdges;
+	}
 
-		if (computationState.isStartState()) {
-			// a computation state is always kept if it refers to a starting state because every
-			// new element can start a new pattern
-			resultingComputationStates.add(computationState);
-		} else {
-			// release the shared entry referenced by the current computation state.
-			sharedBuffer.release(computationState.getState(), computationState.getEvent(), computationState.getTimestamp());
-			// try to remove unnecessary shared buffer entries
-			sharedBuffer.remove(computationState.getState(), computationState.getEvent(), computationState.getTimestamp());
-		}
 
-		return resultingComputationStates;
+	private boolean checkFilterCondition(FilterFunction<T> condition, T event) throws Exception {
+		return condition == null || condition.filter(event);
 	}
 
 	/**
@@ -409,8 +546,8 @@ public class NFA<T> implements Serializable {
 	 * @return Collection of event sequences which end in the given computation state
 	 */
 	private Collection<Map<String, T>> extractPatternMatches(final ComputationState<T> computationState) {
-		Collection<LinkedHashMultimap<State<T>, T>> paths = sharedBuffer.extractPatterns(
-			computationState.getState(),
+		Collection<LinkedHashMultimap<String, T>> paths = sharedBuffer.extractPatterns(
+			computationState.getPreviousState().getName(),
 			computationState.getEvent(),
 			computationState.getTimestamp(),
 			computationState.getVersion());
@@ -420,19 +557,20 @@ public class NFA<T> implements Serializable {
 		TypeSerializer<T> serializer = nonDuplicatingTypeSerializer.getTypeSerializer();
 
 		// generate the correct names from the collection of LinkedHashMultimaps
-		for (LinkedHashMultimap<State<T>, T> path: paths) {
+		for (LinkedHashMultimap<String, T> path: paths) {
 			Map<String, T> resultPath = new HashMap<>();
-			for (State<T> key: path.keySet()) {
+			for (String key: path.keySet()) {
 				int counter = 0;
 				Set<T> events = path.get(key);
 
 				// we iterate over the elements in insertion order
 				for (T event: events) {
 					resultPath.put(
-						events.size() > 1 ? generateStateName(key.getName(), counter): key.getName(),
+						events.size() > 1 ? generateStateName(key, counter): key,
 						// copy the element so that the user can change it
 						serializer.isImmutableType() ? event : serializer.copy(event)
 					);
+					counter++;
 				}
 			}
 
@@ -472,6 +610,7 @@ public class NFA<T> implements Serializable {
 
 	private void writeComputationState(final ComputationState<T> computationState, final ObjectOutputStream oos) throws IOException {
 		oos.writeObject(computationState.getState());
+		oos.writeObject(computationState.getPreviousState());
 		oos.writeLong(computationState.getTimestamp());
 		oos.writeObject(computationState.getVersion());
 		oos.writeLong(computationState.getStartTimestamp());
@@ -490,6 +629,7 @@ public class NFA<T> implements Serializable {
 	@SuppressWarnings("unchecked")
 	private ComputationState<T> readComputationState(ObjectInputStream ois) throws IOException, ClassNotFoundException {
 		final State<T> state = (State<T>)ois.readObject();
+		final State<T> previousState = (State<T>)ois.readObject();
 		final long timestamp = ois.readLong();
 		final DeweyNumber version = (DeweyNumber)ois.readObject();
 		final long startTimestamp = ois.readLong();
@@ -504,7 +644,7 @@ public class NFA<T> implements Serializable {
 			event = null;
 		}
 
-		return new ComputationState<>(state, event, timestamp, version, startTimestamp);
+		return ComputationState.createState(state, previousState, event, timestamp, version, startTimestamp);
 	}
 
 	/**
@@ -629,20 +769,4 @@ public class NFA<T> implements Serializable {
 			return getClass().hashCode();
 		}
 	}
-
-	/**
-	 * Comparator used for imposing the assumption that IGNORE is always the last StateTransition in a state.
-	 */
-	private static final class StateTransitionComparator<T> implements Serializable, Comparator<StateTransition<T>> {
-
-		private static final long serialVersionUID = -2775474935413622278L;
-
-		@Override
-		public int compare(final StateTransition<T> o1, final StateTransition<T> o2) {
-			if (o1.getAction() == o2.getAction()) {
-				return 0;
-			}
-			return o1.getAction() == StateTransitionAction.IGNORE ? 1 : -1;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index b7e288b..e6a8c75 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -212,28 +212,27 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 		SharedBufferEntry<K, V> entry = get(key, value, timestamp);
 
 		if (entry != null) {
-			extractionStates.add(new ExtractionState<K, V>(entry, version, new Stack<SharedBufferEntry<K, V>>()));
+			extractionStates.add(new ExtractionState<>(entry, version, new Stack<SharedBufferEntry<K, V>>()));
 
 			// use a depth first search to reconstruct the previous relations
 			while (!extractionStates.isEmpty()) {
-				ExtractionState<K, V> extractionState = extractionStates.pop();
-				DeweyNumber currentVersion = extractionState.getVersion();
+				final ExtractionState<K, V> extractionState = extractionStates.pop();
 				// current path of the depth first search
-				Stack<SharedBufferEntry<K, V>> currentPath = extractionState.getPath();
+				final Stack<SharedBufferEntry<K, V>> currentPath = extractionState.getPath();
+				final SharedBufferEntry<K, V> currentEntry = extractionState.getEntry();
 
 				// termination criterion
-				if (currentVersion.length() == 1) {
-					LinkedHashMultimap<K, V> completePath = LinkedHashMultimap.create();
+				if (currentEntry == null) {
+					final LinkedHashMultimap<K, V> completePath = LinkedHashMultimap.create();
 
 					while(!currentPath.isEmpty()) {
-						SharedBufferEntry<K, V> currentEntry = currentPath.pop();
+						final SharedBufferEntry<K, V> currentPathEntry = currentPath.pop();
 
-						completePath.put(currentEntry.getKey(), currentEntry.getValueTime().getValue());
+						completePath.put(currentPathEntry.getKey(), currentPathEntry.getValueTime().getValue());
 					}
 
 					result.add(completePath);
 				} else {
-					SharedBufferEntry<K, V> currentEntry = extractionState.getEntry();
 
 					// append state to the path
 					currentPath.push(currentEntry);
@@ -242,17 +241,18 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 					for (SharedBufferEdge<K, V> edge : currentEntry.getEdges()) {
 						// we can only proceed if the current version is compatible to the version
 						// of this previous relation
+						final DeweyNumber currentVersion = extractionState.getVersion();
 						if (currentVersion.isCompatibleWith(edge.getVersion())) {
 							if (firstMatch) {
 								// for the first match we don't have to copy the current path
-								extractionStates.push(new ExtractionState<K, V>(edge.getTarget(), edge.getVersion(), currentPath));
+								extractionStates.push(new ExtractionState<>(edge.getTarget(), edge.getVersion(), currentPath));
 								firstMatch = false;
 							} else {
-								Stack<SharedBufferEntry<K, V>> copy = new Stack<>();
+								final Stack<SharedBufferEntry<K, V>> copy = new Stack<>();
 								copy.addAll(currentPath);
 
 								extractionStates.push(
-									new ExtractionState<K, V>(
+									new ExtractionState<>(
 										edge.getTarget(),
 										edge.getVersion(),
 										copy));
@@ -260,6 +260,7 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 						}
 					}
 				}
+
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
index 50b2cf3..7bcb6ea 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.cep.nfa;
 
+import org.apache.flink.api.common.functions.FilterFunction;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -43,7 +45,7 @@ public class State<T> implements Serializable {
 		this.name = name;
 		this.stateType = stateType;
 
-		stateTransitions = new ArrayList<StateTransition<T>>();
+		stateTransitions = new ArrayList<>();
 	}
 
 	public boolean isFinal() {
@@ -60,8 +62,32 @@ public class State<T> implements Serializable {
 		return stateTransitions;
 	}
 
-	public void addStateTransition(final StateTransition<T> stateTransition) {
-		stateTransitions.add(stateTransition);
+
+	private void addStateTransition(
+		final StateTransitionAction action,
+		final State<T> targetState,
+		final FilterFunction<T> condition) {
+		stateTransitions.add(new StateTransition<T>(this, action, targetState, condition));
+	}
+
+	public void addIgnore(final FilterFunction<T> condition) {
+		addStateTransition(StateTransitionAction.IGNORE, this, condition);
+	}
+
+	public void addIgnore(final State<T> targetState,final FilterFunction<T> condition) {
+		addStateTransition(StateTransitionAction.IGNORE, targetState, condition);
+	}
+
+	public void addTake(final State<T> targetState, final FilterFunction<T> condition) {
+		addStateTransition(StateTransitionAction.TAKE, targetState, condition);
+	}
+
+	public void addProceed(final State<T> targetState, final FilterFunction<T> condition) {
+		addStateTransition(StateTransitionAction.PROCEED, targetState, condition);
+	}
+
+	public void addTake(final FilterFunction<T> condition) {
+		addStateTransition(StateTransitionAction.TAKE, this, condition);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
index 479f28a..e3c7b7a 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
@@ -27,12 +27,18 @@ public class StateTransition<T> implements Serializable {
 	private static final long serialVersionUID = -4825345749997891838L;
 
 	private final StateTransitionAction action;
+	private final State<T> sourceState;
 	private final State<T> targetState;
 	private final FilterFunction<T> condition;
 
-	public StateTransition(final StateTransitionAction action, final State<T> targetState, final FilterFunction<T> condition) {
+	public StateTransition(
+		final State<T> sourceState,
+		final StateTransitionAction action,
+		final State<T> targetState,
+		final FilterFunction<T> condition) {
 		this.action = action;
 		this.targetState = targetState;
+		this.sourceState = sourceState;
 		this.condition = condition;
 	}
 
@@ -44,6 +50,10 @@ public class StateTransition<T> implements Serializable {
 		return targetState;
 	}
 
+	public State<T> getSourceState() {
+		return sourceState;
+	}
+
 	public FilterFunction<T> getCondition() {
 		return condition;
 	}
@@ -55,6 +65,7 @@ public class StateTransition<T> implements Serializable {
 			StateTransition<T> other = (StateTransition<T>) obj;
 
 			return action == other.action &&
+				sourceState.getName().equals(other.sourceState.getName()) &&
 				targetState.getName().equals(other.targetState.getName());
 		} else {
 			return false;
@@ -64,14 +75,17 @@ public class StateTransition<T> implements Serializable {
 	@Override
 	public int hashCode() {
 		// we have to take the name of targetState because the transition might be reflexive
-		return Objects.hash(action, targetState.getName());
+		return Objects.hash(action, targetState.getName(), sourceState.getName());
 	}
 
 	@Override
 	public String toString() {
 		StringBuilder builder = new StringBuilder();
 
-		builder.append("StateTransition(").append(action).append(", ").append(targetState.getName());
+		builder.append("StateTransition(")
+			.append(action).append(", ")
+			.append(sourceState.getName()).append(", ")
+			.append(targetState.getName());
 
 		if (condition != null) {
 			builder.append(", with filter)");

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransitionAction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransitionAction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransitionAction.java
index 70fc7fb..b8ca4e8 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransitionAction.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransitionAction.java
@@ -22,7 +22,7 @@ package org.apache.flink.cep.nfa;
  * Set of actions when doing a state transition from a {@link State} to another.
  */
 public enum StateTransitionAction {
-	TAKE, // take the current event and assign it to the new state
-	IGNORE, // ignore the current event and do the state transition
+	TAKE, // take the current event and assign it to the current state
+	IGNORE, // ignore the current event
 	PROCEED // do the state transition and keep the current event for further processing (epsilon transition)
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/MalformedPatternException.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/MalformedPatternException.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/MalformedPatternException.java
deleted file mode 100644
index a3bb5f4..0000000
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/MalformedPatternException.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cep.nfa.compiler;
-
-/**
- * An exception used to indicate that a {@link org.apache.flink.cep.pattern.Pattern}
- * was not specified correctly.
- */
-public class MalformedPatternException extends RuntimeException {
-
-	private static final long serialVersionUID = 7751134834983361543L;
-
-	public MalformedPatternException(String message) {
-		super(message);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 18ed21f..b476c49 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -22,18 +22,21 @@ import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.State;
-import org.apache.flink.cep.nfa.StateTransition;
-import org.apache.flink.cep.nfa.StateTransitionAction;
+import org.apache.flink.cep.pattern.FilterFunctions;
 import org.apache.flink.cep.pattern.FollowedByPattern;
+import org.apache.flink.cep.pattern.MalformedPatternException;
+import org.apache.flink.cep.pattern.NotFilterFunction;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.Quantifier;
+import org.apache.flink.cep.pattern.Quantifier.QuantifierProperty;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Map;
+import java.util.List;
 import java.util.Set;
 
 /**
@@ -42,7 +45,7 @@ import java.util.Set;
  */
 public class NFACompiler {
 
-	protected final static String BEGINNING_STATE_NAME = "$beginningState$";
+	protected static final String ENDING_STATE_NAME = "$endState$";
 
 	/**
 	 * Compiles the given pattern into a {@link NFA}.
@@ -74,88 +77,288 @@ public class NFACompiler {
 	 */
 	@SuppressWarnings("unchecked")
 	public static <T> NFAFactory<T> compileFactory(
-		Pattern<T, ?> pattern,
-		TypeSerializer<T> inputTypeSerializer,
+		final Pattern<T, ?> pattern,
+		final TypeSerializer<T> inputTypeSerializer,
 		boolean timeoutHandling) {
 		if (pattern == null) {
 			// return a factory for empty NFAs
-			return new NFAFactoryImpl<T>(inputTypeSerializer, 0, Collections.<State<T>>emptyList(), timeoutHandling);
+			return new NFAFactoryImpl<>(inputTypeSerializer, 0, Collections.<State<T>>emptyList(), timeoutHandling);
 		} else {
-			// set of all generated states
-			Map<String, State<T>> states = new HashMap<>();
-			long windowTime;
+			final NFAFactoryCompiler<T> nfaFactoryCompiler = new NFAFactoryCompiler<>(pattern);
+			nfaFactoryCompiler.compileFactory();
+			return new NFAFactoryImpl<>(inputTypeSerializer, nfaFactoryCompiler.getWindowTime(), nfaFactoryCompiler.getStates(), timeoutHandling);
+		}
+	}
+
+	/**
+	 * Converts a {@link Pattern} into graph of {@link State}. It enables sharing of
+	 * compilation state across methods.
+	 *
+	 * @param <T>
+	 */
+	private static class NFAFactoryCompiler<T> {
+
+		private final Set<String> usedNames = new HashSet<>();
+		private final List<State<T>> states = new ArrayList<>();
 
-			// this is used to enforse pattern name uniqueness.
-			Set<String> patternNames = new HashSet<>();
+		private long windowTime = 0;
+		private Pattern<T, ?> currentPattern;
 
-			Pattern<T, ?> succeedingPattern;
-			State<T> succeedingState;
-			Pattern<T, ?> currentPattern = pattern;
+		NFAFactoryCompiler(final Pattern<T, ?> pattern) {
+			this.currentPattern = pattern;
+		}
 
+		/**
+		 * Compiles the given pattern into a {@link NFAFactory}. The NFA factory can be used to create
+		 * multiple NFAs.
+		 */
+		void compileFactory() {
 			// we're traversing the pattern from the end to the beginning --> the first state is the final state
-			State<T> currentState = new State<>(currentPattern.getName(), State.StateType.Final);
-			patternNames.add(currentPattern.getName());
+			State<T> sinkState = createEndingState();
+			// add all the normal states
+			sinkState = createMiddleStates(sinkState);
+			// add the beginning state
+			createStartState(sinkState);
+		}
 
-			states.put(currentPattern.getName(), currentState);
+		List<State<T>> getStates() {
+			return states;
+		}
+
+		long getWindowTime() {
+			return windowTime;
+		}
+
+		/**
+		 * Creates the dummy Final {@link State} of the NFA graph.
+		 * @return dummy Final state
+		 */
+		private State<T> createEndingState() {
+			State<T> endState = new State<>(ENDING_STATE_NAME, State.StateType.Final);
+			states.add(endState);
+			usedNames.add(ENDING_STATE_NAME);
 
 			windowTime = currentPattern.getWindowTime() != null ? currentPattern.getWindowTime().toMilliseconds() : 0L;
+			return endState;
+		}
 
-			while (currentPattern.getPrevious() != null) {
-				succeedingPattern = currentPattern;
-				succeedingState = currentState;
-				currentPattern = currentPattern.getPrevious();
+		/**
+		 * Creates all the states between Start and Final state.
+		 * @param sinkState the state that last state should point to (always the Final state)
+		 * @return the next state after Start in the resulting graph
+		 */
+		private State<T> createMiddleStates(final State<T> sinkState) {
 
-				if (!patternNames.add(currentPattern.getName())) {
-					throw new MalformedPatternException("Duplicate pattern name: " + currentPattern.getName() + ". " +
-						"Pattern names must be unique.");
+			State<T> lastSink = sinkState;
+			while (currentPattern.getPrevious() != null) {
+				checkPatternNameUniqueness();
+
+				State<T> sourceState = new State<>(currentPattern.getName(), State.StateType.Normal);
+				states.add(sourceState);
+				usedNames.add(sourceState.getName());
+
+				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.LOOPING)) {
+					convertToLooping(sourceState, lastSink);
+
+					if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.AT_LEAST_ONE)) {
+						sourceState = createFirstMandatoryStateOfLoop(sourceState, State.StateType.Normal);
+						states.add(sourceState);
+						usedNames.add(sourceState.getName());
+					}
+				} else if (currentPattern.getQuantifier() == Quantifier.TIMES) {
+					sourceState = convertToTimesState(sourceState, lastSink, currentPattern.getTimes());
+				} else {
+					convertToSingletonState(sourceState, lastSink);
 				}
 
-				Time currentWindowTime = currentPattern.getWindowTime();
+				currentPattern = currentPattern.getPrevious();
+				lastSink = sourceState;
 
+				final Time currentWindowTime = currentPattern.getWindowTime();
 				if (currentWindowTime != null && currentWindowTime.toMilliseconds() < windowTime) {
 					// the window time is the global minimum of all window times of each state
 					windowTime = currentWindowTime.toMilliseconds();
 				}
+			}
+
+			return lastSink;
+		}
+
+		private void checkPatternNameUniqueness() {
+			if (usedNames.contains(currentPattern.getName())) {
+				throw new MalformedPatternException(
+					"Duplicate pattern name: " + currentPattern.getName() + ". " +
+					"Pattern names must be unique.");
+			}
+		}
+
+		/**
+		 * Creates the Start {@link State} of the resulting NFA graph.
+		 * @param sinkState the state that Start state should point to (alwyas first state of middle states)
+		 * @return created state
+		 */
+		@SuppressWarnings("unchecked")
+		private State<T> createStartState(State<T> sinkState) {
+			checkPatternNameUniqueness();
+
+			final State<T> beginningState;
+			if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.LOOPING)) {
+				final State<T> loopingState;
+				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.AT_LEAST_ONE)) {
+					loopingState = new State<>(currentPattern.getName(), State.StateType.Normal);
+					beginningState = createFirstMandatoryStateOfLoop(loopingState, State.StateType.Start);
+					states.add(loopingState);
+				} else {
+					loopingState = new State<>(currentPattern.getName(), State.StateType.Start);
+					beginningState = loopingState;
+				}
+				convertToLooping(loopingState, sinkState, true);
+			} else  {
+				if (currentPattern.getQuantifier() == Quantifier.TIMES && currentPattern.getTimes() > 1) {
+					final State<T> timesState = new State<>(currentPattern.getName(), State.StateType.Normal);
+					states.add(timesState);
+					sinkState = convertToTimesState(timesState, sinkState, currentPattern.getTimes() - 1);
+				}
 
-				if (states.containsKey(currentPattern.getName())) {
-					currentState = states.get(currentPattern.getName());
+				beginningState = new State<>(currentPattern.getName(), State.StateType.Start);
+				convertToSingletonState(beginningState, sinkState);
+			}
+
+			states.add(beginningState);
+			usedNames.add(beginningState.getName());
+
+			return beginningState;
+		}
+
+		/**
+		 * Converts the given state into a "complex" state consisting of given number of states with
+		 * same {@link FilterFunction}
+		 *
+		 * @param sourceState the state to be converted
+		 * @param sinkState the state that the converted state should point to
+		 * @param times number of times the state should be copied
+		 * @return the first state of the "complex" state, next state should point to it
+		 */
+		private State<T> convertToTimesState(final State<T> sourceState, final State<T> sinkState, int times) {
+			convertToSingletonState(sourceState, sinkState);
+			State<T> lastSink;
+			State<T> firstState = sourceState;
+			for (int i = 0; i < times - 1; i++) {
+				lastSink = firstState;
+				firstState = new State<>(currentPattern.getName(), State.StateType.Normal);
+				states.add(firstState);
+				convertToSingletonState(firstState, lastSink);
+			}
+			return firstState;
+		}
+
+		/**
+		 * Converts the given state into a simple single state. For an OPTIONAL state it also consists
+		 * of a similar state without the PROCEED edge, so that for each PROCEED transition branches
+		 * in computation state graph  can be created only once.
+		 *
+		 * @param sourceState the state to be converted
+		 * @param sinkState state that the state being converted should point to
+		 */
+		@SuppressWarnings("unchecked")
+		private void convertToSingletonState(final State<T> sourceState, final State<T> sinkState) {
+
+			final FilterFunction<T> currentFilterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
+			final FilterFunction<T> trueFunction = FilterFunctions.trueFunction();
+			sourceState.addTake(sinkState, currentFilterFunction);
+
+			if (currentPattern.getQuantifier() == Quantifier.OPTIONAL) {
+				sourceState.addProceed(sinkState, trueFunction);
+			}
+
+			if (currentPattern instanceof FollowedByPattern) {
+				final State<T> ignoreState;
+				if (currentPattern.getQuantifier() == Quantifier.OPTIONAL) {
+					ignoreState = new State<>(currentPattern.getName(), State.StateType.Normal);
+					ignoreState.addTake(sinkState, currentFilterFunction);
+					states.add(ignoreState);
 				} else {
-					currentState = new State<>(currentPattern.getName(), State.StateType.Normal);
-					states.put(currentState.getName(), currentState);
+					ignoreState = sourceState;
 				}
+				sourceState.addIgnore(ignoreState, trueFunction);
+			}
+		}
 
-				currentState.addStateTransition(new StateTransition<T>(
-					StateTransitionAction.TAKE,
-					succeedingState,
-					(FilterFunction<T>) succeedingPattern.getFilterFunction()));
-
-				if (succeedingPattern instanceof FollowedByPattern) {
-					// the followed by pattern entails a reflexive ignore transition
-					currentState.addStateTransition(new StateTransition<T>(
-						StateTransitionAction.IGNORE,
-						currentState,
-						null
-					));
+		/**
+		 * Patterns with quantifiers AT_LEAST_ONE_* are converted into pair of states: a singleton state and
+		 * looping state. This method creates the first of the two.
+		 *
+		 * @param sinkState the state the newly created state should point to, it should be a looping state
+		 * @param stateType the type of the created state, as the NFA graph can also start wit AT_LEAST_ONE_*
+		 * @return the newly created state
+		 */
+		@SuppressWarnings("unchecked")
+		private State<T> createFirstMandatoryStateOfLoop(final State<T> sinkState, final State.StateType stateType) {
+
+			final FilterFunction<T> currentFilterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
+			final State<T> firstState = new State<>(currentPattern.getName(), stateType);
+
+			firstState.addTake(sinkState, currentFilterFunction);
+			if (currentPattern instanceof FollowedByPattern) {
+				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.EAGER)) {
+					firstState.addIgnore(new NotFilterFunction<>(currentFilterFunction));
+				} else {
+					firstState.addIgnore(FilterFunctions.<T>trueFunction());
 				}
 			}
+			return firstState;
+		}
 
-			// add the beginning state
-			final State<T> beginningState;
+		/**
+		 * Converts the given state into looping one. Looping state is one with TAKE edge to itself and
+		 * PROCEED edge to the sinkState. It also consists of a similar state without the PROCEED edge, so that
+		 * for each PROCEED transition branches in computation state graph  can be created only once.
+		 *
+		 * <p>If this looping state is first of a graph we should treat the {@link Pattern} as {@link FollowedByPattern}
+		 * to enable combinations.
+		 *
+		 * @param sourceState  the state to converted
+		 * @param sinkState    the state that the converted state should point to
+		 * @param isFirstState if the looping state is first of a graph
+		 */
+		@SuppressWarnings("unchecked")
+		private void convertToLooping(final State<T> sourceState, final State<T> sinkState, boolean isFirstState) {
+
+			final FilterFunction<T> filterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
+			final FilterFunction<T> trueFunction = FilterFunctions.<T>trueFunction();
+
+			sourceState.addProceed(sinkState, trueFunction);
+			sourceState.addTake(filterFunction);
+			if (currentPattern instanceof FollowedByPattern || isFirstState) {
+				final State<T> ignoreState = new State<>(
+					currentPattern.getName(),
+					State.StateType.Normal);
+
+
+				final FilterFunction<T> ignoreCondition;
+				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.EAGER)) {
+					ignoreCondition = new NotFilterFunction<>(filterFunction);
+				} else {
+					ignoreCondition = trueFunction;
+				}
 
-			if (states.containsKey(BEGINNING_STATE_NAME)) {
-				beginningState = states.get(BEGINNING_STATE_NAME);
-			} else {
-				beginningState = new State<>(BEGINNING_STATE_NAME, State.StateType.Start);
-				states.put(BEGINNING_STATE_NAME, beginningState);
+				sourceState.addIgnore(ignoreState, ignoreCondition);
+				ignoreState.addTake(sourceState, filterFunction);
+				ignoreState.addIgnore(ignoreState, ignoreCondition);
+				states.add(ignoreState);
 			}
+		}
 
-			beginningState.addStateTransition(new StateTransition<T>(
-				StateTransitionAction.TAKE,
-				currentState,
-				(FilterFunction<T>) currentPattern.getFilterFunction()
-			));
-
-			return new NFAFactoryImpl<T>(inputTypeSerializer, windowTime, new HashSet<>(states.values()), timeoutHandling);
+		/**
+		 * Converts the given state into looping one. Looping state is one with TAKE edge to itself and
+		 * PROCEED edge to the sinkState. It also consists of a similar state without the PROCEED edge, so that
+		 * for each PROCEED transition branches in computation state graph  can be created only once.
+		 *
+		 * @param sourceState the state to converted
+		 * @param sinkState   the state that the converted state should point to
+		 */
+		private void convertToLooping(final State<T> sourceState, final State<T> sinkState) {
+			convertToLooping(sourceState, sinkState, false);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
new file mode 100644
index 0000000..12e58ba
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
@@ -0,0 +1,44 @@
+/*
+ * 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.cep.pattern;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+
+public class FilterFunctions<T> {
+
+	private FilterFunctions() {
+	}
+
+	public static <T> FilterFunction<T> trueFunction()  {
+		return new FilterFunction<T>() {
+			@Override
+			public boolean filter(T value) throws Exception {
+				return true;
+			}
+		};
+	}
+
+	public static <T> FilterFunction<T> falseFunction()  {
+		return new FilterFunction<T>() {
+			@Override
+			public boolean filter(T value) throws Exception {
+				return false;
+			}
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/MalformedPatternException.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/MalformedPatternException.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/MalformedPatternException.java
new file mode 100644
index 0000000..c85f3be
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/MalformedPatternException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cep.pattern;
+
+/**
+ * An exception used to indicate that a {@link org.apache.flink.cep.pattern.Pattern}
+ * was not specified correctly.
+ */
+public class MalformedPatternException extends RuntimeException {
+
+	private static final long serialVersionUID = 7751134834983361543L;
+
+	public MalformedPatternException(String message) {
+		super(message);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
new file mode 100644
index 0000000..a4fc8f5
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cep.pattern;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+
+/**
+ * A filter function which negates filter function.
+ *
+ * @param <T> Type of the element to filter
+ */
+public class NotFilterFunction<T> implements FilterFunction<T> {
+	private static final long serialVersionUID = -2109562093871155005L;
+
+	private final FilterFunction<T> original;
+
+	public NotFilterFunction(final FilterFunction<T> original) {
+		this.original = original;
+	}
+
+	@Override
+	public boolean filter(T value) throws Exception {
+		return !original.filter(value);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
index 7ea675f..7b4d9c7 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.java.ClosureCleaner;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.util.Preconditions;
 
 /**
  * Base class for a pattern definition.
@@ -53,6 +54,10 @@ public class Pattern<T, F extends T> {
 	// window length in which the pattern match has to occur
 	private Time windowTime;
 
+	private Quantifier quantifier = Quantifier.ONE;
+
+	private int times;
+
 	protected Pattern(final String name, final Pattern<T, ? extends T> previous) {
 		this.name = name;
 		this.previous = previous;
@@ -74,6 +79,14 @@ public class Pattern<T, F extends T> {
 		return windowTime;
 	}
 
+	public Quantifier getQuantifier() {
+		return quantifier;
+	}
+
+	public int getTimes() {
+		return times;
+	}
+
 	/**
 	 * Specifies a filter condition which has to be fulfilled by an event in order to be matched.
 	 *
@@ -183,4 +196,106 @@ public class Pattern<T, F extends T> {
 		return new Pattern<X, X>(name, null);
 	}
 
+	/**
+	 * Specifies that this pattern can occur zero or more times(kleene star).
+	 * This means any number of events can be matched in this state.
+	 *
+	 * @return The same pattern with applied Kleene star operator
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> zeroOrMore() {
+		return zeroOrMore(true);
+	}
+
+	/**
+	 * Specifies that this pattern can occur zero or more times(kleene star).
+	 * This means any number of events can be matched in this state.
+	 *
+	 * If eagerness is enabled for a pattern A*B and sequence A1 A2 B will generate patterns:
+	 * B, A1 B and A1 A2 B. If disabled B, A1 B, A2 B and A1 A2 B.
+	 *
+	 * @param eager if true the pattern always consumes earlier events
+	 * @return The same pattern with applied Kleene star operator
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> zeroOrMore(final boolean eager) {
+		checkIfQuantifierApplied();
+		if (eager) {
+			this.quantifier = Quantifier.ZERO_OR_MORE_EAGER;
+		} else {
+			this.quantifier = Quantifier.ZERO_OR_MORE_COMBINATIONS;
+		}
+		return this;
+	}
+
+	/**
+	 * Specifies that this pattern can occur one or more times(kleene star).
+	 * This means at least one and at most infinite number of events can be matched in this state.
+	 *
+	 * @return The same pattern with applied Kleene plus operator
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> oneOrMore() {
+		return oneOrMore(true);
+	}
+
+	/**
+	 * Specifies that this pattern can occur one or more times(kleene star).
+	 * This means at least one and at most infinite number of events can be matched in this state.
+	 *
+	 * If eagerness is enabled for a pattern A+B and sequence A1 A2 B will generate patterns:
+	 * A1 B and A1 A2 B. If disabled A1 B, A2 B and A1 A2 B.
+	 *
+	 * @param eager if true the pattern always consumes earlier events
+	 * @return The same pattern with applied Kleene plus operator
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> oneOrMore(final boolean eager) {
+		checkIfQuantifierApplied();
+		if (eager) {
+			this.quantifier = Quantifier.ONE_OR_MORE_EAGER;
+		} else {
+			this.quantifier = Quantifier.ONE_OR_MORE_COMBINATIONS;
+		}
+		return this;
+	}
+
+	/**
+	 * Specifies that this pattern can occur zero or once.
+	 *
+	 * @return The same pattern with applied Kleene ? operator
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> optional() {
+		checkIfQuantifierApplied();
+		this.quantifier = Quantifier.OPTIONAL;
+		return this;
+	}
+
+	/**
+	 * Specifies exact number of times that this pattern should be matched.
+	 *
+	 * @param times number of times matching event must appear
+	 * @return The same pattern with number of times applied
+	 *
+	 * @throws MalformedPatternException if quantifier already applied
+	 */
+	public Pattern<T, F> times(int times) {
+		checkIfQuantifierApplied();
+		Preconditions.checkArgument(times > 0, "You should give a positive number greater than 0.");
+		this.quantifier = Quantifier.TIMES;
+		this.times = times;
+		return this;
+	}
+
+	private void checkIfQuantifierApplied() {
+		if (this.quantifier != Quantifier.ONE) {
+			throw new MalformedPatternException("Already applied quantifier to this Pattern. Current quantifier is: " + this.quantifier);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.java
new file mode 100644
index 0000000..7abe9bd
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Quantifier.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.cep.pattern;
+
+import java.util.EnumSet;
+
+public enum Quantifier {
+	ONE,
+	ZERO_OR_MORE_EAGER(QuantifierProperty.LOOPING, QuantifierProperty.EAGER),
+	ZERO_OR_MORE_COMBINATIONS(QuantifierProperty.LOOPING),
+	ONE_OR_MORE_EAGER(
+		QuantifierProperty.LOOPING,
+		QuantifierProperty.EAGER,
+		QuantifierProperty.AT_LEAST_ONE),
+	ONE_OR_MORE_COMBINATIONS(QuantifierProperty.LOOPING, QuantifierProperty.AT_LEAST_ONE),
+	TIMES,
+	OPTIONAL;
+
+	private final EnumSet<QuantifierProperty> properties;
+
+	Quantifier(final QuantifierProperty first, final QuantifierProperty... rest) {
+		this.properties = EnumSet.of(first, rest);
+	}
+
+	Quantifier() {
+		this.properties = EnumSet.noneOf(QuantifierProperty.class);
+	}
+
+	public boolean hasProperty(QuantifierProperty property) {
+		return properties.contains(property);
+	}
+
+	public enum QuantifierProperty {
+		LOOPING,
+		EAGER,
+		AT_LEAST_ONE
+	}
+
+}


[07/50] [abbrv] flink git commit: [FLINK-4354] [heartbeat] Add heartbeats between the ResourceManager and TaskExecutor

Posted by fh...@apache.org.
[FLINK-4354] [heartbeat] Add heartbeats between the ResourceManager and TaskExecutor


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

Branch: refs/heads/table-retraction
Commit: 83b99f8a624ddf35deb934b4d4358582657998c6
Parents: fd90672
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Mar 22 12:03:45 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Mar 23 13:58:44 2017 +0100

----------------------------------------------------------------------
 .../heartbeat/TestingHeartbeatManagerImpl.java  | 63 -----------------
 .../TestingHeartbeatManagerSenderImpl.java      | 61 -----------------
 .../heartbeat/TestingHeartbeatServices.java     | 52 --------------
 .../flink/runtime/jobmaster/JobMaster.java      |  4 +-
 .../flink/runtime/minicluster/MiniCluster.java  |  7 +-
 .../resourcemanager/ResourceManager.java        | 71 +++++++++++---------
 .../resourcemanager/ResourceManagerGateway.java |  7 +-
 .../resourcemanager/ResourceManagerRunner.java  |  2 +-
 .../StandaloneResourceManager.java              |  4 +-
 .../runtime/taskexecutor/TaskExecutor.java      | 63 ++++++++++-------
 .../clusterframework/ResourceManagerTest.java   |  6 +-
 .../heartbeat/TestingHeartbeatServices.java     | 52 ++++++++++++++
 .../flink/runtime/jobmaster/JobMasterTest.java  | 10 ++-
 .../resourcemanager/ResourceManagerHATest.java  |  3 +-
 .../ResourceManagerJobMasterTest.java           |  6 +-
 .../ResourceManagerTaskExecutorTest.java        |  7 +-
 .../slotmanager/SlotProtocolTest.java           |  6 +-
 .../taskexecutor/TaskExecutorITCase.java        |  3 +-
 .../runtime/taskexecutor/TaskExecutorTest.java  | 16 +++--
 .../src/test/resources/log4j-test.properties    |  2 +-
 .../yarn/YarnFlinkApplicationMasterRunner.java  |  3 +-
 .../apache/flink/yarn/YarnResourceManager.java  |  4 +-
 22 files changed, 181 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerImpl.java
deleted file mode 100644
index a6e056d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerImpl.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.heartbeat;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.ScheduledExecutor;
-import org.slf4j.Logger;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-
-/**
- * Heartbeat manager implementation which extends {@link HeartbeatManagerImpl} for testing.
- * It overrides the {@link #unmonitorTarget(ResourceID)} to wait for some tests complete
- * when notify heartbeat timeout.
- *
- * @param <I> Type of the incoming heartbeat payload
- * @param <O> Type of the outgoing heartbeat payload
- */
-public class TestingHeartbeatManagerImpl<I, O> extends HeartbeatManagerImpl<I, O> {
-
-	private final CountDownLatch waitLatch;
-
-	public TestingHeartbeatManagerImpl(
-			CountDownLatch waitLatch,
-			long heartbeatTimeoutIntervalMs,
-			ResourceID ownResourceID,
-			HeartbeatListener<I, O> heartbeatListener,
-			Executor executor,
-			ScheduledExecutor scheduledExecutor,
-			Logger log) {
-
-		super(heartbeatTimeoutIntervalMs, ownResourceID, heartbeatListener, executor, scheduledExecutor, log);
-
-		this.waitLatch = waitLatch;
-	}
-
-	@Override
-	public void unmonitorTarget(ResourceID resourceID) {
-		try {
-			waitLatch.await();
-		} catch (InterruptedException ex) {
-			log.error("Unexpected interrupted exception.", ex);
-		}
-
-		super.unmonitorTarget(resourceID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerSenderImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerSenderImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerSenderImpl.java
deleted file mode 100644
index 36f7e96..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatManagerSenderImpl.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.heartbeat;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.ScheduledExecutor;
-import org.slf4j.Logger;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-
-/**
- *
- * @param <I>
- * @param <O>
- */
-public class TestingHeartbeatManagerSenderImpl<I, O> extends HeartbeatManagerSenderImpl<I, O> {
-
-	private final CountDownLatch waitLatch;
-
-	public TestingHeartbeatManagerSenderImpl(
-			CountDownLatch waitLatch,
-			long heartbeatPeriod,
-			long heartbeatTimeout,
-			ResourceID ownResourceID,
-			HeartbeatListener<I, O> heartbeatListener,
-			Executor executor,
-			ScheduledExecutor scheduledExecutor,
-			Logger log) {
-
-		super(heartbeatPeriod, heartbeatTimeout, ownResourceID, heartbeatListener, executor, scheduledExecutor, log);
-
-		this.waitLatch = waitLatch;
-	}
-
-	@Override
-	public void unmonitorTarget(ResourceID resourceID) {
-		try {
-			waitLatch.await();
-		} catch (InterruptedException ex) {
-			log.error("Unexpected interrupted exception.", ex);
-		}
-
-		super.unmonitorTarget(resourceID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
deleted file mode 100644
index e628db5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.heartbeat;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.ScheduledExecutor;
-import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-
-public class TestingHeartbeatServices extends HeartbeatServices {
-
-	private final ScheduledExecutor scheduledExecutorToUse;
-
-	public TestingHeartbeatServices(long heartbeatInterval, long heartbeatTimeout, ScheduledExecutor scheduledExecutorToUse) {
-		super(heartbeatInterval, heartbeatTimeout);
-
-		this.scheduledExecutorToUse = Preconditions.checkNotNull(scheduledExecutorToUse);
-	}
-
-	@Override
-	public <I, O> HeartbeatManager<I, O> createHeartbeatManagerSender(
-		ResourceID resourceId,
-		HeartbeatListener<I, O> heartbeatListener,
-		ScheduledExecutor scheduledExecutor,
-		Logger log) {
-
-		return new HeartbeatManagerSenderImpl<>(
-			heartbeatInterval,
-			heartbeatTimeout,
-			resourceId,
-			heartbeatListener,
-			org.apache.flink.runtime.concurrent.Executors.directExecutor(),
-			scheduledExecutorToUse,
-			log);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index 81fc541..080b48e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -1043,11 +1043,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		@Override
 		public void notifyHeartbeatTimeout(ResourceID resourceID) {
-			log.info("Task manager with id {} heartbeat timed out.", resourceID);
+			log.info("Heartbeat of TaskManager with id {} timed out.", resourceID);
 
 			getSelf().disconnectTaskManager(
 				resourceID,
-				new TimeoutException("Task manager with id " + resourceID + " heartbeat timed out."));
+				new TimeoutException("Heartbeat of TaskManager with id " + resourceID + " timed out."));
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 2cfba7b..9d5f9d1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -241,7 +241,12 @@ public class MiniCluster {
 				// bring up the ResourceManager(s)
 				LOG.info("Starting {} ResourceManger(s)", numResourceManagers);
 				resourceManagerRunners = startResourceManagers(
-						configuration, haServices, heartbeatServices, metricRegistry, numResourceManagers, resourceManagerRpcServices);
+					configuration,
+					haServices,
+					heartbeatServices,
+					metricRegistry,
+					numResourceManagers,
+					resourceManagerRpcServices);
 
 				// bring up the TaskManager(s) for the mini cluster
 				LOG.info("Starting {} TaskManger(s)", numTaskManagers);

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 9a7a790..5467177 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -64,6 +64,7 @@ import org.apache.flink.util.ExceptionUtils;
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -129,8 +130,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private ConcurrentMap<String, InfoMessageListenerRpcGateway> infoMessageListeners;
 
 	public ResourceManager(
-			ResourceID resourceId,
 			RpcService rpcService,
+			ResourceID resourceId,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
 			HeartbeatServices heartbeatServices,
@@ -359,7 +360,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		final ResourceID taskExecutorResourceId,
 		final SlotReport slotReport) {
 
-		if (leaderSessionId.equals(resourceManagerLeaderId)) {
+		if (Objects.equals(leaderSessionId, resourceManagerLeaderId)) {
 			Future<TaskExecutorGateway> taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class);
 
 			return taskExecutorGatewayFuture.handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
@@ -384,7 +385,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 						taskManagerHeartbeatManager.monitorTarget(taskExecutorResourceId, new HeartbeatTarget<Void>() {
 							@Override
 							public void receiveHeartbeat(ResourceID resourceID, Void payload) {
-								// the task manager will not request heartbeat, so this method will never be called currently
+								// the ResourceManager will always send heartbeat requests to the
+								// TaskManager
 							}
 
 							@Override
@@ -394,7 +396,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 						});
 
 						return new TaskExecutorRegistrationSuccess(
-							registration.getInstanceID(), resourceId,
+							registration.getInstanceID(),
+							resourceId,
 							resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds());
 					}
 				}
@@ -607,6 +610,30 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	}
 
 	/**
+	 * This method should be called by the framework once it detects that a currently registered
+	 * task executor has failed.
+	 *
+	 * @param resourceID Id of the TaskManager that has failed.
+	 * @param cause The exception which cause the TaskManager failed.
+	 */
+	protected void closeTaskManagerConnection(final ResourceID resourceID, final Exception cause) {
+		taskManagerHeartbeatManager.unmonitorTarget(resourceID);
+
+		WorkerRegistration<WorkerType> workerRegistration = taskExecutors.remove(resourceID);
+
+		if (workerRegistration != null) {
+			log.info("Task manager {} failed because {}.", resourceID, cause);
+
+			// TODO :: suggest failed task executor to stop itself
+			slotManager.notifyTaskManagerFailure(resourceID);
+
+			workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause);
+		} else {
+			log.debug("Could not find a registered task manager with the process id {}.", resourceID);
+		}
+	}
+
+	/**
 	 * Checks whether the given resource manager leader id is matching the current leader id and
 	 * not null.
 	 *
@@ -756,30 +783,6 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		onFatalErrorAsync(new ResourceManagerException("Received an error from the LeaderElectionService.", exception));
 	}
 
-	/**
-	 * This method should be called by the framework once it detects that a currently registered
-	 * task executor has failed.
-	 *
-	 * @param resourceID Id of the TaskManager that has failed.
-	 * @param cause The exception which cause the TaskManager failed.
-	 */
-	public void closeTaskManagerConnection(final ResourceID resourceID, final Exception cause) {
-		taskManagerHeartbeatManager.unmonitorTarget(resourceID);
-
-		WorkerRegistration<WorkerType> workerRegistration = taskExecutors.remove(resourceID);
-
-		if (workerRegistration != null) {
-			log.info("Task manager {} failed because {}.", resourceID, cause);
-
-			// TODO :: suggest failed task executor to stop itself
-			slotManager.notifyTaskManagerFailure(resourceID);
-
-			workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause);
-		} else {
-			log.debug("Could not find a registered task manager with the process id {}.", resourceID);
-		}
-	}
-
 	// ------------------------------------------------------------------------
 	//  Framework specific behavior
 	// ------------------------------------------------------------------------
@@ -875,11 +878,17 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private class TaskManagerHeartbeatListener implements HeartbeatListener<Void, Void> {
 
 		@Override
-		public void notifyHeartbeatTimeout(ResourceID resourceID) {
+		public void notifyHeartbeatTimeout(final ResourceID resourceID) {
 			log.info("The heartbeat of TaskManager with id {} timed out.", resourceID);
 
-			closeTaskManagerConnection(resourceID, new TimeoutException(
-					"Task manager with id " + resourceID + " heartbeat timed out."));
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					closeTaskManagerConnection(
+						resourceID,
+						new TimeoutException("Task manager with id " + resourceID + " heartbeat timed out."));
+				}
+			});
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
index 7741e0d..cda4a7c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -134,13 +134,12 @@ public interface ResourceManagerGateway extends RpcGateway {
 	/**
 	 * Sends the heartbeat to resource manager from task manager
 	 *
-	 * @param resourceID unique id of the task manager
+	 * @param heartbeatOrigin unique id of the task manager
 	 */
-	void heartbeatFromTaskManager(final ResourceID resourceID);
+	void heartbeatFromTaskManager(final ResourceID heartbeatOrigin);
 
 	/**
-	 * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
-	 * {@link ResourceManager}.
+	 * Disconnects a TaskManager specified by the given resourceID from the {@link ResourceManager}.
 	 *
 	 * @param resourceID identifying the TaskManager to disconnect
 	 * @param cause for the disconnection of the TaskManager

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
index d07e373..3a8baa6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
@@ -69,8 +69,8 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 			rpcService.getScheduledExecutor());
 
 		this.resourceManager = new StandaloneResourceManager(
-			resourceId,
 			rpcService,
+			resourceId,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
 			heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
index e2d6538..fd5a001 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
@@ -38,8 +38,8 @@ import org.apache.flink.runtime.rpc.RpcService;
 public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 
 	public StandaloneResourceManager(
-			ResourceID resourceId,
 			RpcService rpcService,
+			ResourceID resourceId,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
 			HeartbeatServices heartbeatServices,
@@ -48,8 +48,8 @@ public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 		super(
-			resourceId,
 			rpcService,
+			resourceId,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
 			heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index f3e1ff3..4883e7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -696,17 +696,35 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
+	private void establishResourceManagerConnection(ResourceID resourceManagerResourceId) {
+		// monitor the resource manager as heartbeat target
+		resourceManagerHeartbeatManager.monitorTarget(resourceManagerResourceId, new HeartbeatTarget<Void>() {
+			@Override
+			public void receiveHeartbeat(ResourceID resourceID, Void payload) {
+				ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
+				resourceManagerGateway.heartbeatFromTaskManager(resourceID);
+			}
+
+			@Override
+			public void requestHeartbeat(ResourceID resourceID, Void payload) {
+				// the TaskManager won't send heartbeat requests to the ResourceManager
+			}
+		});
+	}
+
 	private void closeResourceManagerConnection(Exception cause) {
-		log.info("Close ResourceManager connection for {}.", cause);
+		validateRunsInMainThread();
 
 		if (isConnectedToResourceManager()) {
+			log.info("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerId(), cause);
+
 			resourceManagerHeartbeatManager.unmonitorTarget(resourceManagerConnection.getResourceManagerId());
 
 			ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
+			resourceManagerGateway.disconnectTaskManager(getResourceID(), cause);
+
 			resourceManagerConnection.close();
 			resourceManagerConnection = null;
-
-			resourceManagerGateway.disconnectTaskManager(getResourceID(), cause);
 		}
 	}
 
@@ -790,7 +808,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 									"and returning them to the ResourceManager.", throwable);
 
 							// We encountered an exception. Free the slots and return them to the RM.
-							for (SlotOffer reservedSlot : reservedSlots) {
+							for (SlotOffer reservedSlot: reservedSlots) {
 								freeSlot(reservedSlot.getAllocationId(), throwable);
 							}
 						}
@@ -841,6 +859,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	private void closeJobManagerConnection(JobID jobId, Exception cause) {
+		validateRunsInMainThread();
+
 		log.info("Close JobManager connection for job {}.", jobId);
 
 		// 1. fail tasks running under this JobID
@@ -1183,21 +1203,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		public void onRegistrationSuccess(TaskExecutorRegistrationSuccess success) {
 			final ResourceID resourceManagerId = success.getResourceManagerId();
 
-			// monitor the resource manager as heartbeat target
-			resourceManagerHeartbeatManager.monitorTarget(resourceManagerId, new HeartbeatTarget<Void>() {
-				@Override
-				public void receiveHeartbeat(ResourceID resourceID, Void payload) {
-					if (isConnectedToResourceManager()) {
-						ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
-						resourceManagerGateway.heartbeatFromTaskManager(resourceID);
+			runAsync(
+				new Runnable() {
+					@Override
+					public void run() {
+						establishResourceManagerConnection(resourceManagerId);
 					}
 				}
-
-				@Override
-				public void requestHeartbeat(ResourceID resourceID, Void payload) {
-					// request heartbeat will never be called on the task manager side
-				}
-			});
+			);
 		}
 
 		@Override
@@ -1277,14 +1290,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					log.info("Job manager with id {} heartbeat timed out.", resourceID);
+					log.info("The heartbeat of JobManager with id {} timed out.", resourceID);
 
 					if (jobManagerConnections.containsKey(resourceID)) {
 						JobManagerConnection jobManagerConnection = jobManagerConnections.get(resourceID);
+
 						if (jobManagerConnection != null) {
 							closeJobManagerConnection(
 								jobManagerConnection.getJobID(),
-								new TimeoutException("Job manager with id " + resourceID + " heartbeat timed out."));
+								new TimeoutException("The heartbeat of JobManager with id " + resourceID + " timed out."));
 						}
 					}
 				}
@@ -1305,16 +1319,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	private class ResourceManagerHeartbeatListener implements HeartbeatListener<Void, Void> {
 
 		@Override
-		public void notifyHeartbeatTimeout(final ResourceID resourceID) {
+		public void notifyHeartbeatTimeout(final ResourceID resourceId) {
 			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					log.info("Resource manager with id {} heartbeat timed out.", resourceID);
+					log.info("The heartbeat of ResourceManager with id {} timed out.", resourceId);
 
-					if (isConnectedToResourceManager() && resourceManagerConnection.getResourceManagerId().equals(resourceID)) {
-						closeResourceManagerConnection(
-								new TimeoutException("Resource manager with id " + resourceID + " heartbeat timed out."));
-					}
+					closeResourceManagerConnection(
+						new TimeoutException(
+							"The heartbeat of ResourceManager with id " + resourceId + " timed out."));
 				}
 			});
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
index e7f2439..72925bb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
@@ -53,6 +53,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.TestingResourceManager;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -76,7 +77,7 @@ import static org.mockito.Mockito.verify;
 /**
  * General tests for the resource manager component.
  */
-public class ResourceManagerTest {
+public class ResourceManagerTest extends TestLogger {
 
 	private static ActorSystem system;
 
@@ -393,8 +394,7 @@ public class ResourceManagerTest {
 
 		try {
 			final StandaloneResourceManager resourceManager = new StandaloneResourceManager(
-				resourceManagerResourceID,
-				rpcService,
+				rpcService, resourceManagerResourceID,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
 				heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
new file mode 100644
index 0000000..e628db5
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
@@ -0,0 +1,52 @@
+/*
+ * 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.runtime.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+
+public class TestingHeartbeatServices extends HeartbeatServices {
+
+	private final ScheduledExecutor scheduledExecutorToUse;
+
+	public TestingHeartbeatServices(long heartbeatInterval, long heartbeatTimeout, ScheduledExecutor scheduledExecutorToUse) {
+		super(heartbeatInterval, heartbeatTimeout);
+
+		this.scheduledExecutorToUse = Preconditions.checkNotNull(scheduledExecutorToUse);
+	}
+
+	@Override
+	public <I, O> HeartbeatManager<I, O> createHeartbeatManagerSender(
+		ResourceID resourceId,
+		HeartbeatListener<I, O> heartbeatListener,
+		ScheduledExecutor scheduledExecutor,
+		Logger log) {
+
+		return new HeartbeatManagerSenderImpl<>(
+			heartbeatInterval,
+			heartbeatTimeout,
+			resourceId,
+			heartbeatListener,
+			org.apache.flink.runtime.concurrent.Executors.directExecutor(),
+			scheduledExecutorToUse,
+			log);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
index 73da244..ee8f51d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
@@ -26,7 +26,8 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoader;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
-import org.apache.flink.runtime.heartbeat.*;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
@@ -49,8 +50,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(BlobLibraryCacheManager.class)
@@ -139,4 +143,6 @@ public class JobMasterTest extends TestLogger {
 			rpc.stopService();
 		}
 	}
+
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
index 39594df..c8e209d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -68,8 +68,7 @@ public class ResourceManagerHATest {
 
 		final ResourceManager resourceManager =
 			new StandaloneResourceManager(
-				rmResourceId,
-				rpcService,
+				rpcService, rmResourceId,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
 				heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
index 0401f9e..32b40ac 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler;
 import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -45,7 +46,7 @@ import java.util.concurrent.TimeUnit;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.*;
 
-public class ResourceManagerJobMasterTest {
+public class ResourceManagerJobMasterTest extends TestLogger {
 
 	private TestingSerialRpcService rpcService;
 
@@ -216,8 +217,7 @@ public class ResourceManagerJobMasterTest {
 			Time.minutes(5L));
 
 		ResourceManager resourceManager = new StandaloneResourceManager(
-			rmResourceId,
-			rpcService,
+			rpcService, rmResourceId,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
 			heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
index 7c811d9..cb0a414 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.taskexecutor.SlotReport;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
+import org.apache.flink.util.TestLogger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -43,7 +44,7 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
-public class ResourceManagerTaskExecutorTest {
+public class ResourceManagerTaskExecutorTest extends TestLogger {
 
 	private TestingSerialRpcService rpcService;
 
@@ -148,7 +149,7 @@ public class ResourceManagerTaskExecutorTest {
 
 	private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
-		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+		HeartbeatServices heartbeatServices = new HeartbeatServices(5L, 5L);
 		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(
@@ -163,8 +164,8 @@ public class ResourceManagerTaskExecutorTest {
 
 		StandaloneResourceManager resourceManager =
 			new StandaloneResourceManager(
-				resourceManagerResourceID,
 				rpcService,
+				resourceManagerResourceID,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
 				heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
index 28ed697..68aff42 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
@@ -237,8 +237,7 @@ public class SlotProtocolTest extends TestLogger {
 
 		ResourceManager<ResourceID> resourceManager =
 			Mockito.spy(new StandaloneResourceManager(
-				rmResourceId,
-				testRpcService,
+				testRpcService, rmResourceId,
 				resourceManagerConfiguration,
 				testingHaServices,
 				heartbeatServices,
@@ -325,8 +324,7 @@ public class SlotProtocolTest extends TestLogger {
 				JobLeaderIdService jobLeaderIdService,
 				FatalErrorHandler fatalErrorHandler) {
 			super(
-				resourceId,
-				rpcService,
+				rpcService, resourceId,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
 				heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
index 4e76486..1789ace 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
@@ -120,8 +120,7 @@ public class TaskExecutorITCase {
 		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 
 		ResourceManager<ResourceID> resourceManager = new StandaloneResourceManager(
-			rmResourceId,
-			rpcService,
+			rpcService, rmResourceId,
 			resourceManagerConfiguration,
 			testingHAServices,
 			heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index d1f6e2e..330d4fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -224,13 +224,19 @@ public class TaskExecutorTest extends TestLogger {
 		ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
 		when(rmGateway.registerTaskExecutor(
 			any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
-			.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(
-					new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId, 10L)));
+			.thenReturn(
+				FlinkCompletableFuture.<RegistrationResponse>completed(
+					new TaskExecutorRegistrationSuccess(
+						new InstanceID(),
+						rmResourceId,
+						10L)));
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		rpc.registerGateway(rmAddress, rmGateway);
 
-		final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+		final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService(
+			null,
+			null);
 		final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
 		haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
@@ -292,11 +298,11 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(rmAddress, rmLeaderId);
 
 			// register resource manager success will trigger monitoring heartbeat target between tm and rm
-			verify(rmGateway).registerTaskExecutor(
+			verify(rmGateway, atLeast(1)).registerTaskExecutor(
 					eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), any(SlotReport.class), any(Time.class));
 
 			// heartbeat timeout should trigger disconnect TaskManager from ResourceManager
-			verify(rmGateway, timeout(heartbeatTimeout * 5)).disconnectTaskManager(eq(taskManagerLocation.getResourceID()), any(TimeoutException.class));
+			verify(rmGateway, timeout(heartbeatTimeout * 50L)).disconnectTaskManager(eq(taskManagerLocation.getResourceID()), any(TimeoutException.class));
 
 			// check if a concurrent error occurred
 			testingFatalErrorHandler.rethrowError();

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-runtime/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties
index 98f136a..7ba1633 100644
--- a/flink-runtime/src/test/resources/log4j-test.properties
+++ b/flink-runtime/src/test/resources/log4j-test.properties
@@ -16,7 +16,7 @@
 # limitations under the License.
 ################################################################################
 
-log4j.rootLogger=INFO, console
+log4j.rootLogger=OFF, console
 
 # -----------------------------------------------------------------------------
 # Console (use 'console')

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index ed672a3..21e6e45 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -199,10 +199,9 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 			commonRpcService.getScheduledExecutor());
 
 		return new YarnResourceManager(
-			ResourceID.generate(),
+			commonRpcService, ResourceID.generate(),
 			config,
 			ENV,
-			commonRpcService,
 			resourceManagerConfiguration,
 			haServices,
 			heartbeatServices,

http://git-wip-us.apache.org/repos/asf/flink/blob/83b99f8a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
index a308079..f8cf275 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -107,10 +107,10 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	final private Map<ResourceProfile, Integer> resourcePriorities = new HashMap<>();
 
 	public YarnResourceManager(
+			RpcService rpcService,
 			ResourceID resourceId,
 			Configuration flinkConfig,
 			Map<String, String> env,
-			RpcService rpcService,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
 			HeartbeatServices heartbeatServices,
@@ -119,8 +119,8 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 		super(
-			resourceId,
 			rpcService,
+			resourceId,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
 			heartbeatServices,


[38/50] [abbrv] flink git commit: [FLINK-6034] [checkpoints] Introduce KeyedStateHandle abstraction for the snapshots in keyed streams

Posted by fh...@apache.org.
[FLINK-6034] [checkpoints] Introduce KeyedStateHandle abstraction for the snapshots in keyed streams


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

Branch: refs/heads/table-retraction
Commit: cd5527417a1cae57073a8855c6c3b88c88c780aa
Parents: 89866a5
Author: xiaogang.sxg <xi...@alibaba-inc.com>
Authored: Thu Mar 23 23:32:15 2017 +0800
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Tue Mar 28 20:05:28 2017 +0200

----------------------------------------------------------------------
 .../state/RocksDBKeyedStateBackend.java         | 46 ++++++++++----
 .../state/RocksDBAsyncSnapshotTest.java         |  3 +-
 .../state/RocksDBStateBackendTest.java          | 21 ++++---
 .../cep/operator/CEPMigration12to13Test.java    | 14 ++---
 .../apache/flink/migration/MigrationUtil.java   | 10 +--
 .../checkpoint/StateAssignmentOperation.java    | 41 ++++++------
 .../flink/runtime/checkpoint/SubtaskState.java  | 14 ++---
 .../savepoint/SavepointV1Serializer.java        | 42 +++++++------
 .../state/AbstractKeyedStateBackend.java        |  2 +-
 .../runtime/state/KeyGroupsStateHandle.java     | 39 ++++--------
 .../flink/runtime/state/KeyedStateHandle.java   | 40 ++++++++++++
 .../state/StateInitializationContextImpl.java   | 28 ++++++++-
 .../StateSnapshotContextSynchronousImpl.java    | 12 ++--
 .../flink/runtime/state/TaskStateHandles.java   | 16 ++---
 .../state/heap/HeapKeyedStateBackend.java       | 46 ++++++++++----
 .../checkpoint/CheckpointCoordinatorTest.java   | 29 +++++----
 .../checkpoint/CheckpointStateRestoreTest.java  |  3 +-
 .../savepoint/MigrationV0ToV1Test.java          | 14 ++++-
 .../KeyedStateCheckpointOutputStreamTest.java   |  4 +-
 .../runtime/state/StateBackendTestBase.java     | 66 ++++++++++----------
 ...pKeyedStateBackendSnapshotMigrationTest.java |  3 +-
 .../api/operators/AbstractStreamOperator.java   |  7 ++-
 .../api/operators/OperatorSnapshotResult.java   | 18 +++---
 .../runtime/tasks/OperatorStateHandles.java     | 14 ++---
 .../streaming/runtime/tasks/StreamTask.java     | 14 ++---
 .../operators/AbstractStreamOperatorTest.java   | 10 +--
 .../operators/OperatorSnapshotResultTest.java   | 10 +--
 .../StateInitializationContextImplTest.java     |  9 +--
 .../tasks/InterruptSensitiveRestoreTest.java    | 17 ++---
 .../streaming/runtime/tasks/StreamTaskTest.java | 14 ++---
 .../util/AbstractStreamOperatorTestHarness.java | 25 ++++----
 .../KeyedOneInputStreamOperatorTestHarness.java | 17 ++---
 .../KeyedTwoInputStreamOperatorTestHarness.java |  3 +-
 33 files changed, 389 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 2ce527f..0407070 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -40,6 +40,7 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.migration.MigrationNamespaceSerializerProxy;
 import org.apache.flink.migration.MigrationUtil;
 import org.apache.flink.migration.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.io.async.AbstractAsyncIOCallable;
 import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
@@ -52,6 +53,7 @@ import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.internal.InternalAggregatingState;
@@ -257,7 +259,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	 * @throws Exception
 	 */
 	@Override
-	public RunnableFuture<KeyGroupsStateHandle> snapshot(
+	public RunnableFuture<KeyedStateHandle> snapshot(
 			final long checkpointId,
 			final long timestamp,
 			final CheckpointStreamFactory streamFactory,
@@ -286,8 +288,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		// implementation of the async IO operation, based on FutureTask
-		AbstractAsyncIOCallable<KeyGroupsStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream> ioCallable =
-				new AbstractAsyncIOCallable<KeyGroupsStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
+		AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream> ioCallable =
+				new AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
 
 					@Override
 					public CheckpointStreamFactory.CheckpointStateOutputStream openIOHandle() throws Exception {
@@ -620,7 +622,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	public void restore(Collection<KeyGroupsStateHandle> restoreState) throws Exception {
+	public void restore(Collection<KeyedStateHandle> restoreState) throws Exception {
 		LOG.info("Initializing RocksDB keyed state backend from snapshot.");
 
 		if (LOG.isDebugEnabled()) {
@@ -669,17 +671,23 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		/**
 		 * Restores all key-groups data that is referenced by the passed state handles.
 		 *
-		 * @param keyGroupsStateHandles List of all key groups state handles that shall be restored.
+		 * @param keyedStateHandles List of all key groups state handles that shall be restored.
 		 * @throws IOException
 		 * @throws ClassNotFoundException
 		 * @throws RocksDBException
 		 */
-		public void doRestore(Collection<KeyGroupsStateHandle> keyGroupsStateHandles)
+		public void doRestore(Collection<KeyedStateHandle> keyedStateHandles)
 				throws IOException, ClassNotFoundException, RocksDBException {
 
-			for (KeyGroupsStateHandle keyGroupsStateHandle : keyGroupsStateHandles) {
-				if (keyGroupsStateHandle != null) {
-					this.currentKeyGroupsStateHandle = keyGroupsStateHandle;
+			for (KeyedStateHandle keyedStateHandle : keyedStateHandles) {
+				if (keyedStateHandle != null) {
+
+					if (!(keyedStateHandle instanceof KeyGroupsStateHandle)) {
+						throw new IllegalStateException("Unexpected state handle type, " +
+								"expected: " + KeyGroupsStateHandle.class +
+								", but found: " + keyedStateHandle.getClass());
+					}
+					this.currentKeyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle;
 					restoreKeyGroupsInStateHandle();
 				}
 			}
@@ -761,6 +769,12 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private void restoreKVStateData() throws IOException, RocksDBException {
 			//for all key-groups in the current state handle...
 			for (Tuple2<Integer, Long> keyGroupOffset : currentKeyGroupsStateHandle.getGroupRangeOffsets()) {
+				int keyGroup = keyGroupOffset.f0;
+
+				// Check that restored key groups all belong to the backend
+				Preconditions.checkState(rocksDBKeyedStateBackend.getKeyGroupRange().contains(keyGroup),
+					"The key group must belong to the backend");
+
 				long offset = keyGroupOffset.f1;
 				//not empty key-group?
 				if (0L != offset) {
@@ -1143,15 +1157,25 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	 * For backwards compatibility, remove again later!
 	 */
 	@Deprecated
-	private void restoreOldSavepointKeyedState(Collection<KeyGroupsStateHandle> restoreState) throws Exception {
+	private void restoreOldSavepointKeyedState(Collection<KeyedStateHandle> restoreState) throws Exception {
 
 		if (restoreState.isEmpty()) {
 			return;
 		}
 
 		Preconditions.checkState(1 == restoreState.size(), "Only one element expected here.");
+
+		KeyedStateHandle keyedStateHandle = restoreState.iterator().next();
+		if (!(keyedStateHandle instanceof MigrationKeyGroupStateHandle)) {
+			throw new IllegalStateException("Unexpected state handle type, " +
+					"expected: " + MigrationKeyGroupStateHandle.class +
+					", but found: " + keyedStateHandle.getClass());
+		}
+
+		MigrationKeyGroupStateHandle keyGroupStateHandle = (MigrationKeyGroupStateHandle) keyedStateHandle;
+
 		HashMap<String, RocksDBStateBackend.FinalFullyAsyncSnapshot> namedStates;
-		try (FSDataInputStream inputStream = restoreState.iterator().next().openInputStream()) {
+		try (FSDataInputStream inputStream = keyGroupStateHandle.openInputStream()) {
 			namedStates = InstantiationUtil.deserializeObject(inputStream, userCodeClassLoader);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
index 90de7a6..ffe2ce2 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
@@ -42,6 +42,7 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory;
@@ -343,7 +344,7 @@ public class RocksDBAsyncSnapshotTest {
 			StringSerializer.INSTANCE,
 			new ValueStateDescriptor<>("foobar", String.class));
 
-		RunnableFuture<KeyGroupsStateHandle> snapshotFuture = keyedStateBackend.snapshot(
+		RunnableFuture<KeyedStateHandle> snapshotFuture = keyedStateBackend.snapshot(
 			checkpointId, timestamp, checkpointStreamFactory, CheckpointOptions.forFullCheckpoint());
 
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
index 708613b..d95a9b4 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.StateBackendTestBase;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
@@ -172,7 +173,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testRunningSnapshotAfterBackendClosed() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
 			CheckpointOptions.forFullCheckpoint());
 
 		RocksDB spyDB = keyedStateBackend.db;
@@ -210,7 +211,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testReleasingSnapshotAfterBackendClosed() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
 			CheckpointOptions.forFullCheckpoint());
 
 		RocksDB spyDB = keyedStateBackend.db;
@@ -239,7 +240,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testDismissingSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
 		snapshot.cancel(true);
 		verifyRocksObjectsReleased();
 	}
@@ -247,7 +248,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testDismissingSnapshotNotRunnable() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
 		snapshot.cancel(true);
 		Thread asyncSnapshotThread = new Thread(snapshot);
 		asyncSnapshotThread.start();
@@ -264,7 +265,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testCompletingSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
 		Thread asyncSnapshotThread = new Thread(snapshot);
 		asyncSnapshotThread.start();
 		waiter.await(); // wait for snapshot to run
@@ -272,10 +273,10 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 		runStateUpdates();
 		blocker.trigger(); // allow checkpointing to start writing
 		waiter.await(); // wait for snapshot stream writing to run
-		KeyGroupsStateHandle keyGroupsStateHandle = snapshot.get();
-		assertNotNull(keyGroupsStateHandle);
-		assertTrue(keyGroupsStateHandle.getStateSize() > 0);
-		assertEquals(2, keyGroupsStateHandle.getNumberOfKeyGroups());
+		KeyedStateHandle keyedStateHandle = snapshot.get();
+		assertNotNull(keyedStateHandle);
+		assertTrue(keyedStateHandle.getStateSize() > 0);
+		assertEquals(2, keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
 		assertTrue(testStreamFactory.getLastCreatedStream().isClosed());
 		asyncSnapshotThread.join();
 		verifyRocksObjectsReleased();
@@ -284,7 +285,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	@Test
 	public void testCancelRunningSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyGroupsStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
 		Thread asyncSnapshotThread = new Thread(snapshot);
 		asyncSnapshotThread.start();
 		waiter.await(); // wait for snapshot to run

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
index f230bbc..dbe4230 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
@@ -26,7 +26,7 @@ import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.streaming.api.watermark.Watermark;
@@ -128,8 +128,8 @@ public class CEPMigration12to13Test {
 		final OperatorStateHandles snapshot = new OperatorStateHandles(
 			(int) ois.readObject(),
 			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject()
 		);
@@ -243,8 +243,8 @@ public class CEPMigration12to13Test {
 		final OperatorStateHandles snapshot = new OperatorStateHandles(
 			(int) ois.readObject(),
 			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject()
 		);
@@ -363,8 +363,8 @@ public class CEPMigration12to13Test {
 		final OperatorStateHandles snapshot = new OperatorStateHandles(
 			(int) ois.readObject(),
 			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
-			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
+			(Collection<KeyedStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject(),
 			(Collection<OperatorStateHandle>) ois.readObject()
 		);

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
index 9427f72..a4e3a2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
+++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
@@ -19,17 +19,17 @@
 package org.apache.flink.migration;
 
 import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import java.util.Collection;
 
 public class MigrationUtil {
 
 	@SuppressWarnings("deprecation")
-	public static boolean isOldSavepointKeyedState(Collection<KeyGroupsStateHandle> keyGroupsStateHandles) {
-		return (keyGroupsStateHandles != null)
-				&& (keyGroupsStateHandles.size() == 1)
-				&& (keyGroupsStateHandles.iterator().next() instanceof MigrationKeyGroupStateHandle);
+	public static boolean isOldSavepointKeyedState(Collection<KeyedStateHandle> keyedStateHandles) {
+		return (keyedStateHandles != null)
+				&& (keyedStateHandles.size() == 1)
+				&& (keyedStateHandles.iterator().next() instanceof MigrationKeyGroupStateHandle);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
index 3fda430..ac70e1a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateHandles;
@@ -160,8 +161,8 @@ public class StateAssignmentOperation {
 		@SuppressWarnings("unchecked")
 		List<OperatorStateHandle>[] parallelOpStatesStream = new List[chainLength];
 
-		List<KeyGroupsStateHandle> parallelKeyedStatesBackend = new ArrayList<>(oldParallelism);
-		List<KeyGroupsStateHandle> parallelKeyedStateStream = new ArrayList<>(oldParallelism);
+		List<KeyedStateHandle> parallelKeyedStatesBackend = new ArrayList<>(oldParallelism);
+		List<KeyedStateHandle> parallelKeyedStateStream = new ArrayList<>(oldParallelism);
 
 		for (int p = 0; p < oldParallelism; ++p) {
 			SubtaskState subtaskState = taskState.getState(p);
@@ -173,12 +174,12 @@ public class StateAssignmentOperation {
 				collectParallelStatesByChainOperator(
 						parallelOpStatesStream, subtaskState.getRawOperatorState());
 
-				KeyGroupsStateHandle keyedStateBackend = subtaskState.getManagedKeyedState();
+				KeyedStateHandle keyedStateBackend = subtaskState.getManagedKeyedState();
 				if (null != keyedStateBackend) {
 					parallelKeyedStatesBackend.add(keyedStateBackend);
 				}
 
-				KeyGroupsStateHandle keyedStateStream = subtaskState.getRawKeyedState();
+				KeyedStateHandle keyedStateStream = subtaskState.getRawKeyedState();
 				if (null != keyedStateStream) {
 					parallelKeyedStateStream.add(keyedStateStream);
 				}
@@ -252,13 +253,13 @@ public class StateAssignmentOperation {
 					.getTaskVertices()[subTaskIdx]
 					.getCurrentExecutionAttempt();
 
-			List<KeyGroupsStateHandle> newKeyedStatesBackend;
-			List<KeyGroupsStateHandle> newKeyedStateStream;
+			List<KeyedStateHandle> newKeyedStatesBackend;
+			List<KeyedStateHandle> newKeyedStateStream;
 			if (oldParallelism == newParallelism) {
 				SubtaskState subtaskState = taskState.getState(subTaskIdx);
 				if (subtaskState != null) {
-					KeyGroupsStateHandle oldKeyedStatesBackend = subtaskState.getManagedKeyedState();
-					KeyGroupsStateHandle oldKeyedStatesStream = subtaskState.getRawKeyedState();
+					KeyedStateHandle oldKeyedStatesBackend = subtaskState.getManagedKeyedState();
+					KeyedStateHandle oldKeyedStatesStream = subtaskState.getRawKeyedState();
 					newKeyedStatesBackend = oldKeyedStatesBackend != null ? Collections.singletonList(
 							oldKeyedStatesBackend) : null;
 					newKeyedStateStream = oldKeyedStatesStream != null ? Collections.singletonList(
@@ -269,8 +270,8 @@ public class StateAssignmentOperation {
 				}
 			} else {
 				KeyGroupRange subtaskKeyGroupIds = keyGroupPartitions.get(subTaskIdx);
-				newKeyedStatesBackend = getKeyGroupsStateHandles(parallelKeyedStatesBackend, subtaskKeyGroupIds);
-				newKeyedStateStream = getKeyGroupsStateHandles(parallelKeyedStateStream, subtaskKeyGroupIds);
+				newKeyedStatesBackend = getKeyedStateHandles(parallelKeyedStatesBackend, subtaskKeyGroupIds);
+				newKeyedStateStream = getKeyedStateHandles(parallelKeyedStateStream, subtaskKeyGroupIds);
 			}
 
 			TaskStateHandles taskStateHandles = new TaskStateHandles(
@@ -290,19 +291,21 @@ public class StateAssignmentOperation {
 	 * <p>
 	 * <p>This is publicly visible to be used in tests.
 	 */
-	public static List<KeyGroupsStateHandle> getKeyGroupsStateHandles(
-			Collection<KeyGroupsStateHandle> allKeyGroupsHandles,
-			KeyGroupRange subtaskKeyGroupIds) {
+	public static List<KeyedStateHandle> getKeyedStateHandles(
+			Collection<? extends KeyedStateHandle> keyedStateHandles,
+			KeyGroupRange subtaskKeyGroupRange) {
 
-		List<KeyGroupsStateHandle> subtaskKeyGroupStates = new ArrayList<>();
+		List<KeyedStateHandle> subtaskKeyedStateHandles = new ArrayList<>();
 
-		for (KeyGroupsStateHandle storedKeyGroup : allKeyGroupsHandles) {
-			KeyGroupsStateHandle intersection = storedKeyGroup.getKeyGroupIntersection(subtaskKeyGroupIds);
-			if (intersection.getNumberOfKeyGroups() > 0) {
-				subtaskKeyGroupStates.add(intersection);
+		for (KeyedStateHandle keyedStateHandle : keyedStateHandles) {
+			KeyedStateHandle intersectedKeyedStateHandle = keyedStateHandle.getIntersection(subtaskKeyGroupRange);
+
+			if (intersectedKeyedStateHandle != null) {
+				subtaskKeyedStateHandles.add(intersectedKeyedStateHandle);
 			}
 		}
-		return subtaskKeyGroupStates;
+
+		return subtaskKeyedStateHandles;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
index 1393e32..9e195b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.runtime.state.ChainedStateHandle;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StateUtil;
@@ -56,12 +56,12 @@ public class SubtaskState implements StateObject {
 	/**
 	 * Snapshot from {@link org.apache.flink.runtime.state.KeyedStateBackend}.
 	 */
-	private final KeyGroupsStateHandle managedKeyedState;
+	private final KeyedStateHandle managedKeyedState;
 
 	/**
 	 * Snapshot written using {@link org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream}.
 	 */
-	private final KeyGroupsStateHandle rawKeyedState;
+	private final KeyedStateHandle rawKeyedState;
 
 	/**
 	 * The state size. This is also part of the deserialized state handle.
@@ -74,8 +74,8 @@ public class SubtaskState implements StateObject {
 			ChainedStateHandle<StreamStateHandle> legacyOperatorState,
 			ChainedStateHandle<OperatorStateHandle> managedOperatorState,
 			ChainedStateHandle<OperatorStateHandle> rawOperatorState,
-			KeyGroupsStateHandle managedKeyedState,
-			KeyGroupsStateHandle rawKeyedState) {
+			KeyedStateHandle managedKeyedState,
+			KeyedStateHandle rawKeyedState) {
 
 		this.legacyOperatorState = checkNotNull(legacyOperatorState, "State");
 		this.managedOperatorState = managedOperatorState;
@@ -114,11 +114,11 @@ public class SubtaskState implements StateObject {
 		return rawOperatorState;
 	}
 
-	public KeyGroupsStateHandle getManagedKeyedState() {
+	public KeyedStateHandle getManagedKeyedState() {
 		return managedKeyedState;
 	}
 
-	public KeyGroupsStateHandle getRawKeyedState() {
+	public KeyedStateHandle getRawKeyedState() {
 		return rawKeyedState;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
index ba1949a..44461d8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
@@ -155,11 +156,11 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 			serializeOperatorStateHandle(stateHandle, dos);
 		}
 
-		KeyGroupsStateHandle keyedStateBackend = subtaskState.getManagedKeyedState();
-		serializeKeyGroupStateHandle(keyedStateBackend, dos);
+		KeyedStateHandle keyedStateBackend = subtaskState.getManagedKeyedState();
+		serializeKeyedStateHandle(keyedStateBackend, dos);
 
-		KeyGroupsStateHandle keyedStateStream = subtaskState.getRawKeyedState();
-		serializeKeyGroupStateHandle(keyedStateStream, dos);
+		KeyedStateHandle keyedStateStream = subtaskState.getRawKeyedState();
+		serializeKeyedStateHandle(keyedStateStream, dos);
 	}
 
 	private static SubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException {
@@ -188,9 +189,9 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 			operatorStateStream.add(streamStateHandle);
 		}
 
-		KeyGroupsStateHandle keyedStateBackend = deserializeKeyGroupStateHandle(dis);
+		KeyedStateHandle keyedStateBackend = deserializeKeyedStateHandle(dis);
 
-		KeyGroupsStateHandle keyedStateStream = deserializeKeyGroupStateHandle(dis);
+		KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis);
 
 		ChainedStateHandle<StreamStateHandle> nonPartitionableStateChain =
 				new ChainedStateHandle<>(nonPartitionableState);
@@ -209,23 +210,27 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 				keyedStateStream);
 	}
 
-	private static void serializeKeyGroupStateHandle(
-			KeyGroupsStateHandle stateHandle, DataOutputStream dos) throws IOException {
+	private static void serializeKeyedStateHandle(
+			KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException {
+
+		if (stateHandle == null) {
+			dos.writeByte(NULL_HANDLE);
+		} else if (stateHandle instanceof KeyGroupsStateHandle) {
+			KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) stateHandle;
 
-		if (stateHandle != null) {
 			dos.writeByte(KEY_GROUPS_HANDLE);
-			dos.writeInt(stateHandle.getGroupRangeOffsets().getKeyGroupRange().getStartKeyGroup());
-			dos.writeInt(stateHandle.getNumberOfKeyGroups());
-			for (int keyGroup : stateHandle.keyGroups()) {
-				dos.writeLong(stateHandle.getOffsetForKeyGroup(keyGroup));
+			dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getStartKeyGroup());
+			dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
+			for (int keyGroup : keyGroupsStateHandle.getKeyGroupRange()) {
+				dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup));
 			}
-			serializeStreamStateHandle(stateHandle.getDelegateStateHandle(), dos);
+			serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos);
 		} else {
-			dos.writeByte(NULL_HANDLE);
+			throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass());
 		}
 	}
 
-	private static KeyGroupsStateHandle deserializeKeyGroupStateHandle(DataInputStream dis) throws IOException {
+	private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException {
 		final int type = dis.readByte();
 		if (NULL_HANDLE == type) {
 			return null;
@@ -237,11 +242,12 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 			for (int i = 0; i < numKeyGroups; ++i) {
 				offsets[i] = dis.readLong();
 			}
-			KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(keyGroupRange, offsets);
+			KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(
+				keyGroupRange, offsets);
 			StreamStateHandle stateHandle = deserializeStreamStateHandle(dis);
 			return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle);
 		} else {
-			throw new IllegalStateException("Reading invalid KeyGroupsStateHandle, type: " + type);
+			throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index e6e7b23..e86f1f8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -61,7 +61,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * @param <K> Type of the key by which state is keyed.
  */
 public abstract class AbstractKeyedStateBackend<K>
-		implements KeyedStateBackend<K>, Snapshotable<KeyGroupsStateHandle>, Closeable {
+		implements KeyedStateBackend<K>, Snapshotable<KeyedStateHandle>, Closeable {
 
 	/** {@link TypeSerializer} for our key. */
 	protected final TypeSerializer<K> keySerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
index b454e42..bad7fd4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
@@ -29,7 +29,7 @@ import java.io.IOException;
  * consists of a range of key group snapshots. A key group is subset of the available
  * key space. The key groups are identified by their key group indices.
  */
-public class KeyGroupsStateHandle implements StreamStateHandle {
+public class KeyGroupsStateHandle implements StreamStateHandle, KeyedStateHandle {
 
 	private static final long serialVersionUID = -8070326169926626355L;
 
@@ -54,20 +54,18 @@ public class KeyGroupsStateHandle implements StreamStateHandle {
 
 	/**
 	 *
-	 * @return iterable over the key-group range for the key-group state referenced by this handle
+	 * @return the internal key-group range to offsets metadata
 	 */
-	public Iterable<Integer> keyGroups() {
-		return groupRangeOffsets.getKeyGroupRange();
+	public KeyGroupRangeOffsets getGroupRangeOffsets() {
+		return groupRangeOffsets;
 	}
 
-
 	/**
 	 *
-	 * @param keyGroupId the id of a key-group
-	 * @return true if the provided key-group id is contained in the key-group range of this handle
+	 * @return The handle to the actual states
 	 */
-	public boolean containsKeyGroup(int keyGroupId) {
-		return groupRangeOffsets.getKeyGroupRange().contains(keyGroupId);
+	public StreamStateHandle getDelegateStateHandle() {
+		return stateHandle;
 	}
 
 	/**
@@ -85,24 +83,13 @@ public class KeyGroupsStateHandle implements StreamStateHandle {
 	 * @return key-group state over a range that is the intersection between this handle's key-group range and the
 	 *          provided key-group range.
 	 */
-	public KeyGroupsStateHandle getKeyGroupIntersection(KeyGroupRange keyGroupRange) {
+	public KeyGroupsStateHandle getIntersection(KeyGroupRange keyGroupRange) {
 		return new KeyGroupsStateHandle(groupRangeOffsets.getIntersection(keyGroupRange), stateHandle);
 	}
 
-	/**
-	 *
-	 * @return the internal key-group range to offsets metadata
-	 */
-	public KeyGroupRangeOffsets getGroupRangeOffsets() {
-		return groupRangeOffsets;
-	}
-
-	/**
-	 *
-	 * @return number of key-groups in the key-group range of this handle
-	 */
-	public int getNumberOfKeyGroups() {
-		return groupRangeOffsets.getKeyGroupRange().getNumberOfKeyGroups();
+	@Override
+	public KeyGroupRange getKeyGroupRange() {
+		return groupRangeOffsets.getKeyGroupRange();
 	}
 
 	@Override
@@ -120,10 +107,6 @@ public class KeyGroupsStateHandle implements StreamStateHandle {
 		return stateHandle.openInputStream();
 	}
 
-	public StreamStateHandle getDelegateStateHandle() {
-		return stateHandle;
-	}
-
 	@Override
 	public boolean equals(Object o) {
 		if (this == o) {

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
new file mode 100644
index 0000000..dc9c97d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
@@ -0,0 +1,40 @@
+/*
+ * 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.runtime.state;
+
+/**
+ * Base for the handles of the checkpointed states in keyed streams. When
+ * recovering from failures, the handle will be passed to all tasks whose key
+ * group ranges overlap with it.
+ */
+public interface KeyedStateHandle extends StateObject {
+
+	/**
+	 * Returns the range of the key groups contained in the state.
+	 */
+	KeyGroupRange getKeyGroupRange();
+
+	/**
+	 * Returns a state over a range that is the intersection between this
+	 * handle's key-group range and the provided key-group range.
+	 *
+	 * @param keyGroupRange The key group range to intersect with
+	 */
+	KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java
index 886d214..d82af72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java
@@ -27,9 +27,11 @@ import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.List;
 import java.util.NoSuchElementException;
 
 /**
@@ -55,7 +57,7 @@ public class StateInitializationContextImpl implements StateInitializationContex
 			boolean restored,
 			OperatorStateStore operatorStateStore,
 			KeyedStateStore keyedStateStore,
-			Collection<KeyGroupsStateHandle> keyGroupsStateHandles,
+			Collection<KeyedStateHandle> keyedStateHandles,
 			Collection<OperatorStateHandle> operatorStateHandles,
 			CloseableRegistry closableRegistry) {
 
@@ -64,7 +66,7 @@ public class StateInitializationContextImpl implements StateInitializationContex
 		this.operatorStateStore = operatorStateStore;
 		this.keyedStateStore = keyedStateStore;
 		this.operatorStateHandles = operatorStateHandles;
-		this.keyGroupsStateHandles = keyGroupsStateHandles;
+		this.keyGroupsStateHandles = transform(keyedStateHandles);
 
 		this.keyedStateIterable = keyGroupsStateHandles == null ?
 				null
@@ -136,6 +138,26 @@ public class StateInitializationContextImpl implements StateInitializationContex
 		IOUtils.closeQuietly(closableRegistry);
 	}
 
+	private static Collection<KeyGroupsStateHandle> transform(Collection<KeyedStateHandle> keyedStateHandles) {
+		if (keyedStateHandles == null) {
+			return null;
+		}
+
+		List<KeyGroupsStateHandle> keyGroupsStateHandles = new ArrayList<>();
+
+		for (KeyedStateHandle keyedStateHandle : keyedStateHandles) {
+			if (! (keyedStateHandle instanceof KeyGroupsStateHandle)) {
+				throw new IllegalStateException("Unexpected state handle type, " +
+					"expected: " + KeyGroupsStateHandle.class +
+					", but found: " + keyedStateHandle.getClass() + ".");
+			}
+
+			keyGroupsStateHandles.add((KeyGroupsStateHandle) keyedStateHandle);
+		}
+
+		return keyGroupsStateHandles;
+	}
+
 	private static class KeyGroupStreamIterator
 			extends AbstractStateStreamIterator<KeyGroupStatePartitionStreamProvider, KeyGroupsStateHandle> {
 
@@ -159,7 +181,7 @@ public class StateInitializationContextImpl implements StateInitializationContex
 
 			while (stateHandleIterator.hasNext()) {
 				currentStateHandle = stateHandleIterator.next();
-				if (currentStateHandle.getNumberOfKeyGroups() > 0) {
+				if (currentStateHandle.getKeyGroupRange().getNumberOfKeyGroups() > 0) {
 					currentOffsetsIterator = currentStateHandle.getGroupRangeOffsets().iterator();
 
 					return true;

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java
index 96edccb..5db0138 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java
@@ -109,15 +109,17 @@ public class StateSnapshotContextSynchronousImpl implements StateSnapshotContext
 		return operatorStateCheckpointOutputStream;
 	}
 
-	public RunnableFuture<KeyGroupsStateHandle> getKeyedStateStreamFuture() throws IOException {
-		return closeAndUnregisterStreamToObtainStateHandle(keyedStateCheckpointOutputStream);
+	public RunnableFuture<KeyedStateHandle> getKeyedStateStreamFuture() throws IOException {
+		KeyGroupsStateHandle keyGroupsStateHandle = closeAndUnregisterStreamToObtainStateHandle(keyedStateCheckpointOutputStream);
+		return new DoneFuture<KeyedStateHandle>(keyGroupsStateHandle);
 	}
 
 	public RunnableFuture<OperatorStateHandle> getOperatorStateStreamFuture() throws IOException {
-		return closeAndUnregisterStreamToObtainStateHandle(operatorStateCheckpointOutputStream);
+		OperatorStateHandle operatorStateHandle = closeAndUnregisterStreamToObtainStateHandle(operatorStateCheckpointOutputStream);
+		return new DoneFuture<>(operatorStateHandle);
 	}
 
-	private <T extends StreamStateHandle> RunnableFuture<T> closeAndUnregisterStreamToObtainStateHandle(
+	private <T extends StreamStateHandle> T closeAndUnregisterStreamToObtainStateHandle(
 			NonClosingCheckpointOutputStream<T> stream) throws IOException {
 		if (null == stream) {
 			return null;
@@ -126,7 +128,7 @@ public class StateSnapshotContextSynchronousImpl implements StateSnapshotContext
 		closableRegistry.unregisterClosable(stream.getDelegate());
 
 		// for now we only support synchronous writing
-		return new DoneFuture<>(stream.closeAndGetHandle());
+		return stream.closeAndGetHandle();
 	}
 
 	private <T extends StreamStateHandle> void closeAndUnregisterStream(NonClosingCheckpointOutputStream<T> stream) throws IOException {

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java
index 417a9dd..450413a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java
@@ -40,10 +40,10 @@ public class TaskStateHandles implements Serializable {
 	private final ChainedStateHandle<StreamStateHandle> legacyOperatorState;
 
 	/** Collection of handles which represent the managed keyed state of the head operator */
-	private final Collection<KeyGroupsStateHandle> managedKeyedState;
+	private final Collection<KeyedStateHandle> managedKeyedState;
 
 	/** Collection of handles which represent the raw/streamed keyed state of the head operator */
-	private final Collection<KeyGroupsStateHandle> rawKeyedState;
+	private final Collection<KeyedStateHandle> rawKeyedState;
 
 	/** Outer list represents the operator chain, each collection holds handles for managed state of a single operator */
 	private final List<Collection<OperatorStateHandle>> managedOperatorState;
@@ -67,8 +67,8 @@ public class TaskStateHandles implements Serializable {
 			ChainedStateHandle<StreamStateHandle> legacyOperatorState,
 			List<Collection<OperatorStateHandle>> managedOperatorState,
 			List<Collection<OperatorStateHandle>> rawOperatorState,
-			Collection<KeyGroupsStateHandle> managedKeyedState,
-			Collection<KeyGroupsStateHandle> rawKeyedState) {
+			Collection<KeyedStateHandle> managedKeyedState,
+			Collection<KeyedStateHandle> rawKeyedState) {
 
 		this.legacyOperatorState = legacyOperatorState;
 		this.managedKeyedState = managedKeyedState;
@@ -82,11 +82,11 @@ public class TaskStateHandles implements Serializable {
 		return legacyOperatorState;
 	}
 
-	public Collection<KeyGroupsStateHandle> getManagedKeyedState() {
+	public Collection<KeyedStateHandle> getManagedKeyedState() {
 		return managedKeyedState;
 	}
 
-	public Collection<KeyGroupsStateHandle> getRawKeyedState() {
+	public Collection<KeyedStateHandle> getRawKeyedState() {
 		return rawKeyedState;
 	}
 
@@ -110,8 +110,8 @@ public class TaskStateHandles implements Serializable {
 		return out;
 	}
 
-	private static List<KeyGroupsStateHandle> transform(KeyGroupsStateHandle in) {
-		return in == null ? Collections.<KeyGroupsStateHandle>emptyList() : Collections.singletonList(in);
+	private static <T> List<T> transform(T in) {
+		return in == null ? Collections.<T>emptyList() : Collections.singletonList(in);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index 46ec5c2..a332d7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -39,6 +39,7 @@ import org.apache.flink.migration.runtime.state.KvStateSnapshot;
 import org.apache.flink.migration.runtime.state.memory.MigrationRestoreSnapshot;
 import org.apache.flink.runtime.io.async.AbstractAsyncIOCallable;
 import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
+import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
@@ -50,6 +51,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.internal.InternalAggregatingState;
@@ -223,7 +225,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	@SuppressWarnings("unchecked")
-	public  RunnableFuture<KeyGroupsStateHandle> snapshot(
+	public  RunnableFuture<KeyedStateHandle> snapshot(
 			final long checkpointId,
 			final long timestamp,
 			final CheckpointStreamFactory streamFactory,
@@ -267,8 +269,8 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		//--------------------------------------------------- this becomes the end of sync part
 
 		// implementation of the async IO operation, based on FutureTask
-		final AbstractAsyncIOCallable<KeyGroupsStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream> ioCallable =
-				new AbstractAsyncIOCallable<KeyGroupsStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
+		final AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream> ioCallable =
+				new AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
 
 					AtomicBoolean open = new AtomicBoolean(false);
 
@@ -340,7 +342,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					}
 				};
 
-		AsyncStoppableTaskWithCallback<KeyGroupsStateHandle> task = AsyncStoppableTaskWithCallback.from(ioCallable);
+		AsyncStoppableTaskWithCallback<KeyedStateHandle> task = AsyncStoppableTaskWithCallback.from(ioCallable);
 
 		if (!asynchronousSnapshots) {
 			task.run();
@@ -354,7 +356,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@SuppressWarnings("deprecation")
 	@Override
-	public void restore(Collection<KeyGroupsStateHandle> restoredState) throws Exception {
+	public void restore(Collection<KeyedStateHandle> restoredState) throws Exception {
 		LOG.info("Initializing heap keyed state backend from snapshot.");
 
 		if (LOG.isDebugEnabled()) {
@@ -369,19 +371,26 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@SuppressWarnings({"unchecked"})
-	private void restorePartitionedState(Collection<KeyGroupsStateHandle> state) throws Exception {
+	private void restorePartitionedState(Collection<KeyedStateHandle> state) throws Exception {
 
 		final Map<Integer, String> kvStatesById = new HashMap<>();
 		int numRegisteredKvStates = 0;
 		stateTables.clear();
 
-		for (KeyGroupsStateHandle keyGroupsHandle : state) {
+		for (KeyedStateHandle keyedStateHandle : state) {
 
-			if (keyGroupsHandle == null) {
+			if (keyedStateHandle == null) {
 				continue;
 			}
 
-			FSDataInputStream fsDataInputStream = keyGroupsHandle.openInputStream();
+			if (!(keyedStateHandle instanceof KeyGroupsStateHandle)) {
+				throw new IllegalStateException("Unexpected state handle type, " +
+						"expected: " + KeyGroupsStateHandle.class +
+						", but found: " + keyedStateHandle.getClass());
+			}
+
+			KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle;
+			FSDataInputStream fsDataInputStream = keyGroupsStateHandle.openInputStream();
 			cancelStreamRegistry.registerClosable(fsDataInputStream);
 
 			try {
@@ -412,9 +421,13 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					}
 				}
 
-				for (Tuple2<Integer, Long> groupOffset : keyGroupsHandle.getGroupRangeOffsets()) {
+				for (Tuple2<Integer, Long> groupOffset : keyGroupsStateHandle.getGroupRangeOffsets()) {
 					int keyGroupIndex = groupOffset.f0;
 					long offset = groupOffset.f1;
+
+					// Check that restored key groups all belong to the backend.
+					Preconditions.checkState(keyGroupRange.contains(keyGroupIndex), "The key group must belong to the backend.");
+
 					fsDataInputStream.seek(offset);
 
 					int writtenKeyGroupIndex = inView.readInt();
@@ -449,7 +462,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	@SuppressWarnings({"unchecked", "rawtypes", "DeprecatedIsStillUsed"})
 	@Deprecated
 	private void restoreOldSavepointKeyedState(
-			Collection<KeyGroupsStateHandle> stateHandles) throws IOException, ClassNotFoundException {
+			Collection<KeyedStateHandle> stateHandles) throws IOException, ClassNotFoundException {
 
 		if (stateHandles.isEmpty()) {
 			return;
@@ -457,8 +470,17 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 		Preconditions.checkState(1 == stateHandles.size(), "Only one element expected here.");
 
+		KeyedStateHandle keyedStateHandle = stateHandles.iterator().next();
+		if (!(keyedStateHandle instanceof MigrationKeyGroupStateHandle)) {
+			throw new IllegalStateException("Unexpected state handle type, " +
+					"expected: " + MigrationKeyGroupStateHandle.class +
+					", but found " + keyedStateHandle.getClass());
+		}
+
+		MigrationKeyGroupStateHandle keyGroupStateHandle = (MigrationKeyGroupStateHandle) keyedStateHandle;
+
 		HashMap<String, KvStateSnapshot<K, ?, ?, ?>> namedStates;
-		try (FSDataInputStream inputStream = stateHandles.iterator().next().openInputStream()) {
+		try (FSDataInputStream inputStream = keyGroupStateHandle.openInputStream()) {
 			namedStates = InstantiationUtil.deserializeObject(inputStream, userCodeClassLoader);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
index d8bba59..117c70d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
@@ -41,6 +41,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateHandles;
@@ -2346,13 +2347,13 @@ public class CheckpointCoordinatorTest {
 			ChainedStateHandle<StreamStateHandle> operatorState = taskStateHandles.getLegacyOperatorState();
 			List<Collection<OperatorStateHandle>> opStateBackend = taskStateHandles.getManagedOperatorState();
 			List<Collection<OperatorStateHandle>> opStateRaw = taskStateHandles.getRawOperatorState();
-			Collection<KeyGroupsStateHandle> keyGroupStateBackend = taskStateHandles.getManagedKeyedState();
-			Collection<KeyGroupsStateHandle> keyGroupStateRaw = taskStateHandles.getRawKeyedState();
+			Collection<KeyedStateHandle> keyedStateBackend = taskStateHandles.getManagedKeyedState();
+			Collection<KeyedStateHandle> keyGroupStateRaw = taskStateHandles.getRawKeyedState();
 
 			actualOpStatesBackend.add(opStateBackend);
 			actualOpStatesRaw.add(opStateRaw);
 			assertNull(operatorState);
-			compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyGroupStateBackend);
+			compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend);
 			compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw);
 		}
 		comparePartitionableState(expectedOpStatesBackend, actualOpStatesBackend);
@@ -2690,32 +2691,38 @@ public class CheckpointCoordinatorTest {
 
 			KeyGroupsStateHandle expectPartitionedKeyGroupState = generateKeyGroupState(
 					jobVertexID, keyGroupPartitions.get(i), false);
-			Collection<KeyGroupsStateHandle> actualPartitionedKeyGroupState = taskStateHandles.getManagedKeyedState();
+			Collection<KeyedStateHandle> actualPartitionedKeyGroupState = taskStateHandles.getManagedKeyedState();
 			compareKeyedState(Collections.singletonList(expectPartitionedKeyGroupState), actualPartitionedKeyGroupState);
 		}
 	}
 
 	public static void compareKeyedState(
 			Collection<KeyGroupsStateHandle> expectPartitionedKeyGroupState,
-			Collection<KeyGroupsStateHandle> actualPartitionedKeyGroupState) throws Exception {
+			Collection<? extends KeyedStateHandle> actualPartitionedKeyGroupState) throws Exception {
 
 		KeyGroupsStateHandle expectedHeadOpKeyGroupStateHandle = expectPartitionedKeyGroupState.iterator().next();
-		int expectedTotalKeyGroups = expectedHeadOpKeyGroupStateHandle.getNumberOfKeyGroups();
+		int expectedTotalKeyGroups = expectedHeadOpKeyGroupStateHandle.getKeyGroupRange().getNumberOfKeyGroups();
 		int actualTotalKeyGroups = 0;
-		for(KeyGroupsStateHandle keyGroupsStateHandle: actualPartitionedKeyGroupState) {
-			actualTotalKeyGroups += keyGroupsStateHandle.getNumberOfKeyGroups();
+		for(KeyedStateHandle keyedStateHandle: actualPartitionedKeyGroupState) {
+			assertTrue(keyedStateHandle instanceof KeyGroupsStateHandle);
+
+			actualTotalKeyGroups += keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups();
 		}
 
 		assertEquals(expectedTotalKeyGroups, actualTotalKeyGroups);
 
 		try (FSDataInputStream inputStream = expectedHeadOpKeyGroupStateHandle.openInputStream()) {
-			for (int groupId : expectedHeadOpKeyGroupStateHandle.keyGroups()) {
+			for (int groupId : expectedHeadOpKeyGroupStateHandle.getKeyGroupRange()) {
 				long offset = expectedHeadOpKeyGroupStateHandle.getOffsetForKeyGroup(groupId);
 				inputStream.seek(offset);
 				int expectedKeyGroupState =
 						InstantiationUtil.deserializeObject(inputStream, Thread.currentThread().getContextClassLoader());
-				for (KeyGroupsStateHandle oneActualKeyGroupStateHandle : actualPartitionedKeyGroupState) {
-					if (oneActualKeyGroupStateHandle.containsKeyGroup(groupId)) {
+				for (KeyedStateHandle oneActualKeyedStateHandle : actualPartitionedKeyGroupState) {
+
+					assertTrue(oneActualKeyedStateHandle instanceof KeyGroupsStateHandle);
+
+					KeyGroupsStateHandle oneActualKeyGroupStateHandle = (KeyGroupsStateHandle) oneActualKeyedStateHandle;
+					if (oneActualKeyGroupStateHandle.getKeyGroupRange().contains(groupId)) {
 						long actualOffset = oneActualKeyGroupStateHandle.getOffsetForKeyGroup(groupId);
 						try (FSDataInputStream actualInputStream = oneActualKeyGroupStateHandle.openInputStream()) {
 							actualInputStream.seek(actualOffset);

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
index 18b07eb..7e0a7c1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateHandles;
@@ -68,7 +69,7 @@ public class CheckpointStateRestoreTest {
 			final ChainedStateHandle<StreamStateHandle> serializedState = CheckpointCoordinatorTest.generateChainedStateHandle(new SerializableObject());
 			KeyGroupRange keyGroupRange = KeyGroupRange.of(0,0);
 			List<SerializableObject> testStates = Collections.singletonList(new SerializableObject());
-			final KeyGroupsStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates);
+			final KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates);
 
 			final JobID jid = new JobID();
 			final JobVertexID statefulId = new JobVertexID();

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
index 6ab8620..1ecb2e3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
@@ -37,6 +37,7 @@ import org.apache.flink.runtime.checkpoint.TaskState;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
@@ -58,6 +59,7 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 @SuppressWarnings("deprecation")
 public class MigrationV0ToV1Test {
@@ -154,9 +156,15 @@ public class MigrationV0ToV1Test {
 					}
 
 					//check keyed state
-					KeyGroupsStateHandle keyGroupsStateHandle = subtaskState.getManagedKeyedState();
+					KeyedStateHandle keyedStateHandle = subtaskState.getManagedKeyedState();
+
 					if (t % 3 != 0) {
-						assertEquals(1, keyGroupsStateHandle.getNumberOfKeyGroups());
+
+						assertTrue(keyedStateHandle instanceof KeyGroupsStateHandle);
+
+						KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle;
+
+						assertEquals(1, keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
 						assertEquals(p, keyGroupsStateHandle.getGroupRangeOffsets().getKeyGroupRange().getStartKeyGroup());
 
 						ByteStreamStateHandle stateHandle =
@@ -172,7 +180,7 @@ public class MigrationV0ToV1Test {
 							assertEquals(p, data[1]);
 						}
 					} else {
-						assertEquals(null, keyGroupsStateHandle);
+						assertEquals(null, keyedStateHandle);
 					}
 				}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyedStateCheckpointOutputStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyedStateCheckpointOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyedStateCheckpointOutputStreamTest.java
index 0c4ed74..cee0b02 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyedStateCheckpointOutputStreamTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyedStateCheckpointOutputStreamTest.java
@@ -135,7 +135,7 @@ public class KeyedStateCheckpointOutputStreamTest {
 		int count = 0;
 		try (FSDataInputStream in = fullHandle.openInputStream()) {
 			DataInputView div = new DataInputViewStreamWrapper(in);
-			for (int kg : fullHandle.keyGroups()) {
+			for (int kg : fullHandle.getKeyGroupRange()) {
 				long off = fullHandle.getOffsetForKeyGroup(kg);
 				if (off >= 0) {
 					in.seek(off);
@@ -152,7 +152,7 @@ public class KeyedStateCheckpointOutputStreamTest {
 		int count = 0;
 		try (FSDataInputStream in = fullHandle.openInputStream()) {
 			DataInputView div = new DataInputViewStreamWrapper(in);
-			for (int kg : fullHandle.keyGroups()) {
+			for (int kg : fullHandle.getKeyGroupRange()) {
 				long off = fullHandle.getOffsetForKeyGroup(kg);
 				in.seek(off);
 				Assert.assertEquals(kg, div.readInt());

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index 22bb715..ccc1eae 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -143,13 +143,13 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				env.getTaskKvStateRegistry());
 	}
 
-	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(TypeSerializer<K> keySerializer, KeyGroupsStateHandle state) throws Exception {
+	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(TypeSerializer<K> keySerializer, KeyedStateHandle state) throws Exception {
 		return restoreKeyedBackend(keySerializer, state, new DummyEnvironment("test", 1, 0));
 	}
 
 	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(
 			TypeSerializer<K> keySerializer,
-			KeyGroupsStateHandle state,
+			KeyedStateHandle state,
 			Environment env) throws Exception {
 		return restoreKeyedBackend(
 				keySerializer,
@@ -163,7 +163,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			TypeSerializer<K> keySerializer,
 			int numberOfKeyGroups,
 			KeyGroupRange keyGroupRange,
-			List<KeyGroupsStateHandle> state,
+			List<KeyedStateHandle> state,
 			Environment env) throws Exception {
 
 		AbstractKeyedStateBackend<K> backend = getStateBackend().createKeyedStateBackend(
@@ -436,7 +436,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		backend.setCurrentKey(2);
 		state.update(new TestPojo("u2", 2));
 
-		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+		KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
@@ -497,7 +497,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		backend.setCurrentKey(2);
 		state.update(new TestPojo("u2", 2));
 
-		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+		KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
@@ -524,7 +524,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		// update to test state backends that eagerly serialize, such as RocksDB
 		state.update(new TestPojo("u1", 11));
 
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
@@ -585,7 +585,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		backend.setCurrentKey(2);
 		state.update(new TestPojo("u2", 2));
 
-		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+		KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
@@ -611,7 +611,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		// update to test state backends that eagerly serialize, such as RocksDB
 		state.update(new TestPojo("u1", 11));
 
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
@@ -670,7 +670,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals("1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// make some more modifications
 		backend.setCurrentKey(1);
@@ -681,7 +681,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.update("u3");
 
 		// draw another snapshot
-		KeyGroupsStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// validate the original state
 		backend.setCurrentKey(1);
@@ -880,7 +880,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals(13, (int) state2.value());
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		backend.dispose();
 		backend = restoreKeyedBackend(
@@ -952,7 +952,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals(42L, (long) state.value());
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		backend.dispose();
 		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
@@ -997,7 +997,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals("1", joiner.join(getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// make some more modifications
 		backend.setCurrentKey(1);
@@ -1008,7 +1008,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.add("u3");
 
 		// draw another snapshot
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// validate the original state
 		backend.setCurrentKey(1);
@@ -1091,7 +1091,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals("1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// make some more modifications
 		backend.setCurrentKey(1);
@@ -1102,7 +1102,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.add("u3");
 
 		// draw another snapshot
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// validate the original state
 		backend.setCurrentKey(1);
@@ -1188,7 +1188,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		assertEquals("Fold-Initial:,1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// make some more modifications
 		backend.setCurrentKey(1);
@@ -1200,7 +1200,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.add(103);
 
 		// draw another snapshot
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// validate the original state
 		backend.setCurrentKey(1);
@@ -1287,7 +1287,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
 
 		// draw a snapshot
-		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// make some more modifications
 		backend.setCurrentKey(1);
@@ -1299,7 +1299,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.putAll(new HashMap<Integer, String>() {{ put(1031, "1031"); put(1032, "1032"); }});
 
 		// draw another snapshot
-		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		// validate the original state
 		backend.setCurrentKey(1);
@@ -1606,13 +1606,13 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		state.update("ShouldBeInSecondHalf");
 
 
-		KeyGroupsStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(0, 0, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(0, 0, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-		List<KeyGroupsStateHandle> firstHalfKeyGroupStates = StateAssignmentOperation.getKeyGroupsStateHandles(
+		List<KeyedStateHandle> firstHalfKeyGroupStates = StateAssignmentOperation.getKeyedStateHandles(
 				Collections.singletonList(snapshot),
 				KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex(MAX_PARALLELISM, 2, 0));
 
-		List<KeyGroupsStateHandle> secondHalfKeyGroupStates = StateAssignmentOperation.getKeyGroupsStateHandles(
+		List<KeyedStateHandle> secondHalfKeyGroupStates = StateAssignmentOperation.getKeyedStateHandles(
 				Collections.singletonList(snapshot),
 				KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex(MAX_PARALLELISM, 2, 1));
 
@@ -1672,7 +1672,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			state.update("2");
 
 			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+			KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 			backend.dispose();
 			// restore the first snapshot and validate it
@@ -1723,7 +1723,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			state.add("2");
 
 			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+			KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 			backend.dispose();
 			// restore the first snapshot and validate it
@@ -1776,7 +1776,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			state.add("2");
 
 			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+			KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 			backend.dispose();
 			// restore the first snapshot and validate it
@@ -1827,7 +1827,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			state.put("2", "Second");
 
 			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+			KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 			backend.dispose();
 			// restore the first snapshot and validate it
@@ -2093,7 +2093,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				eq(env.getJobID()), eq(env.getJobVertexId()), eq(expectedKeyGroupRange), eq("banana"), any(KvStateID.class));
 
 
-		KeyGroupsStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		KeyedStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
 		backend.dispose();
 
@@ -2124,7 +2124,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			ListStateDescriptor<String> kvId = new ListStateDescriptor<>("id", String.class);
 
 			// draw a snapshot
-			KeyGroupsStateHandle snapshot =
+			KeyedStateHandle snapshot =
 					FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 1, streamFactory, CheckpointOptions.forFullCheckpoint()));
 			assertNull(snapshot);
 			backend.dispose();
@@ -2152,7 +2152,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		streamFactory.setWaiterLatch(waiter);
 
 		AbstractKeyedStateBackend<Integer> backend = null;
-		KeyGroupsStateHandle stateHandle = null;
+		KeyedStateHandle stateHandle = null;
 
 		try {
 			backend = createKeyedBackend(IntSerializer.INSTANCE);
@@ -2167,7 +2167,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				valueState.update(i);
 			}
 
-			RunnableFuture<KeyGroupsStateHandle> snapshot =
+			RunnableFuture<KeyedStateHandle> snapshot =
 					backend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forFullCheckpoint());
 			Thread runner = new Thread(snapshot);
 			runner.start();
@@ -2249,7 +2249,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				valueState.update(i);
 			}
 
-			RunnableFuture<KeyGroupsStateHandle> snapshot =
+			RunnableFuture<KeyedStateHandle> snapshot =
 					backend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forFullCheckpoint());
 
 			Thread runner = new Thread(snapshot);
@@ -2367,7 +2367,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		}
 	}
 
-	private KeyGroupsStateHandle runSnapshot(RunnableFuture<KeyGroupsStateHandle> snapshotRunnableFuture) throws Exception {
+	private KeyedStateHandle runSnapshot(RunnableFuture<KeyedStateHandle> snapshotRunnableFuture) throws Exception {
 		if(!snapshotRunnableFuture.isDone()) {
 			Thread runner = new Thread(snapshotRunnableFuture);
 			runner.start();

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java
index da0666a..3754d63 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
@@ -63,7 +64,7 @@ public class HeapKeyedStateBackendSnapshotMigrationTest extends HeapStateBackend
 			try (BufferedInputStream bis = new BufferedInputStream((new FileInputStream(resource.getFile())))) {
 				stateHandle = InstantiationUtil.deserializeObject(bis, Thread.currentThread().getContextClassLoader());
 			}
-			keyedBackend.restore(Collections.singleton(stateHandle));
+			keyedBackend.restore(Collections.<KeyedStateHandle>singleton(stateHandle));
 			final ListStateDescriptor<Long> stateDescr = new ListStateDescriptor<>("my-state", Long.class);
 			stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index e40a59b..a6a89b5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.streaming.api.operators;
 
-import java.io.IOException;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
 import org.apache.flink.annotation.PublicEvolving;
@@ -47,9 +46,9 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
 import org.apache.flink.runtime.state.KeyGroupsList;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateBackend;
 import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateInitializationContext;
@@ -70,6 +69,7 @@ import org.apache.flink.util.OutputTag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
@@ -198,7 +198,7 @@ public abstract class AbstractStreamOperator<OUT>
 	@Override
 	public final void initializeState(OperatorStateHandles stateHandles) throws Exception {
 
-		Collection<KeyGroupsStateHandle> keyedStateHandlesRaw = null;
+		Collection<KeyedStateHandle> keyedStateHandlesRaw = null;
 		Collection<OperatorStateHandle> operatorStateHandlesRaw = null;
 		Collection<OperatorStateHandle> operatorStateHandlesBackend = null;
 
@@ -473,6 +473,7 @@ public abstract class AbstractStreamOperator<OUT>
 			// and then initialize the timer services
 			for (KeyGroupStatePartitionStreamProvider streamProvider : context.getRawKeyedStateInputs()) {
 				int keyGroupIdx = streamProvider.getKeyGroupId();
+
 				checkArgument(localKeyGroupRange.contains(keyGroupIdx),
 					"Key Group " + keyGroupIdx + " does not belong to the local range.");
 


[27/50] [abbrv] flink git commit: [FLINK-5929] Allow Access to Per-Window State in ProcessWindowFunction

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java
index d4fefa2..6f34607 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java
@@ -21,9 +21,11 @@ package org.apache.flink.streaming.api.operators.windowing.functions;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.KeyedStateStore;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
 import org.apache.flink.streaming.util.functions.StreamingFunctionUtils;
 import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.RichProcessAllWindowFunction;
@@ -31,6 +33,7 @@ import org.apache.flink.streaming.api.functions.windowing.RichProcessWindowFunct
 import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
 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.InternalAggregateProcessAllWindowFunction;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalAggregateProcessWindowFunction;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableAllWindowFunction;
@@ -41,6 +44,7 @@ import org.apache.flink.streaming.runtime.operators.windowing.functions.Internal
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueProcessAllWindowFunction;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueProcessWindowFunction;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
 import org.apache.flink.util.Collector;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 import org.junit.Test;
@@ -56,7 +60,11 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.collection.IsMapContaining.hasEntry;
 import static org.hamcrest.core.AllOf.allOf;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.anyObject;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 
 public class InternalWindowFunctionTest {
 
@@ -93,7 +101,9 @@ public class InternalWindowFunctionTest {
 		Iterable<Long> i = (Iterable<Long>)mock(Iterable.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
 
-		windowFunction.apply(((byte)0), w, i, c);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
+
+		windowFunction.process(((byte)0), w, ctx, i, c);
 		verify(mock).apply(w, i, c);
 
 		// check close
@@ -134,7 +144,8 @@ public class InternalWindowFunctionTest {
 		Iterable<Long> i = (Iterable<Long>)mock(Iterable.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
 
-		windowFunction.apply(((byte)0), w, i, c);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
+		windowFunction.process(((byte)0), w, ctx, i, c);
 		verify(mock).process((ProcessAllWindowFunctionMock.Context) anyObject(), eq(i), eq(c));
 
 		// check close
@@ -175,7 +186,8 @@ public class InternalWindowFunctionTest {
 		Iterable<Long> i = (Iterable<Long>)mock(Iterable.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
 
-		windowFunction.apply(42L, w, i, c);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
+		windowFunction.process(42L, w, ctx, i, c);
 		verify(mock).apply(eq(42L), eq(w), eq(i), eq(c));
 
 		// check close
@@ -215,8 +227,9 @@ public class InternalWindowFunctionTest {
 		TimeWindow w = mock(TimeWindow.class);
 		Iterable<Long> i = (Iterable<Long>)mock(Iterable.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(42L, w, i, c);
+		windowFunction.process(42L, w, ctx, i, c);
 		verify(mock).process(eq(42L), (ProcessWindowFunctionMock.Context) anyObject(), eq(i), eq(c));
 
 		// check close
@@ -256,8 +269,9 @@ public class InternalWindowFunctionTest {
 		// check apply
 		TimeWindow w = mock(TimeWindow.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(42L, w, 23L, c);
+		windowFunction.process(42L, w, ctx, 23L, c);
 		verify(mock).apply(eq(42L), eq(w), (Iterable<Long>)argThat(IsIterableContainingInOrder.contains(23L)), eq(c));
 
 		// check close
@@ -297,8 +311,9 @@ public class InternalWindowFunctionTest {
 		// check apply
 		TimeWindow w = mock(TimeWindow.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(((byte)0), w, 23L, c);
+		windowFunction.process(((byte)0), w, ctx, 23L, c);
 		verify(mock).apply(eq(w), (Iterable<Long>)argThat(IsIterableContainingInOrder.contains(23L)), eq(c));
 
 		// check close
@@ -338,8 +353,9 @@ public class InternalWindowFunctionTest {
 		// check apply
 		TimeWindow w = mock(TimeWindow.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(((byte)0), w, 23L, c);
+		windowFunction.process(((byte)0), w, ctx, 23L, c);
 		verify(mock).process((ProcessAllWindowFunctionMock.Context) anyObject(), (Iterable<Long>)argThat(IsIterableContainingInOrder.contains(23L)), eq(c));
 
 		// check close
@@ -378,8 +394,9 @@ public class InternalWindowFunctionTest {
 		// check apply
 		TimeWindow w = mock(TimeWindow.class);
 		Collector<String> c = (Collector<String>) mock(Collector.class);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(42L, w, 23L, c);
+		windowFunction.process(42L, w, ctx,23L, c);
 		verify(mock).process(eq(42L), (ProcessWindowFunctionMock.Context) anyObject(), (Iterable<Long>)argThat(IsIterableContainingInOrder.contains(23L)), eq(c));
 
 		// check close
@@ -450,8 +467,9 @@ public class InternalWindowFunctionTest {
 		List<Long> args = new LinkedList<>();
 		args.add(23L);
 		args.add(24L);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(42L, w, args, c);
+		windowFunction.process(42L, w, ctx, args, c);
 		verify(mock).process(
 				eq(42L),
 				(AggregateProcessWindowFunctionMock.Context) anyObject(),
@@ -528,8 +546,9 @@ public class InternalWindowFunctionTest {
 		List<Long> args = new LinkedList<>();
 		args.add(23L);
 		args.add(24L);
+		InternalWindowFunction.InternalWindowContext ctx = mock(InternalWindowFunction.InternalWindowContext.class);
 
-		windowFunction.apply(((byte)0), w, args, c);
+		windowFunction.process(((byte)0), w, ctx, args, c);
 		verify(mock).process(
 				(AggregateProcessAllWindowFunctionMock.Context) anyObject(),
 				(Iterable) argThat(containsInAnyOrder(allOf(
@@ -552,7 +571,9 @@ public class InternalWindowFunctionTest {
 		public void setOutputType(TypeInformation<String> outTypeInfo, ExecutionConfig executionConfig) { }
 
 		@Override
-		public void process(Long aLong, Context context, Iterable<Long> input, Collector<String> out) throws Exception { }
+		public void process(Long aLong, ProcessWindowFunction<Long, String, Long, TimeWindow>.Context context, Iterable<Long> elements, Collector<String> out) throws Exception {
+
+		}
 	}
 
 	public static class AggregateProcessWindowFunctionMock
@@ -565,7 +586,9 @@ public class InternalWindowFunctionTest {
 		public void setOutputType(TypeInformation<String> outTypeInfo, ExecutionConfig executionConfig) { }
 
 		@Override
-		public void process(Long aLong, Context context, Iterable<Map<Long, Long>> input, Collector<String> out) throws Exception { }
+		public void process(Long aLong, ProcessWindowFunction<Map<Long, Long>, String, Long, TimeWindow>.Context context, Iterable<Map<Long, Long>> elements, Collector<String> out) throws Exception {
+
+		}
 	}
 
 	public static class AggregateProcessAllWindowFunctionMock

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
index 11508c5..ff1cbdf 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
@@ -118,9 +118,9 @@ public class RegularWindowOperatorContractTest extends WindowOperatorContractTes
 
 		testHarness.processElement(new StreamRecord<>(1, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(1), anyTimeWindow(), anyInternalWindowContext(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(1), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(1), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), eq(3), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -182,9 +182,9 @@ public class RegularWindowOperatorContractTest extends WindowOperatorContractTes
 
 		testHarness.processElement(new StreamRecord<>(1, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(1), anyTimeWindow(), anyInternalWindowContext(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(1), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(1), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), eq(3), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
index 8aae46a..faab505 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -163,6 +163,10 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		return Mockito.any();
 	}
 
+	static InternalWindowFunction.InternalWindowContext anyInternalWindowContext() {
+		return Mockito.any();
+	}
+
 	static Trigger.OnMergeContext anyOnMergeContext() {
 		return Mockito.any();
 	}
@@ -408,9 +412,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq((new TimeWindow(0, 2))), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 	}
 
 	@Test
@@ -455,9 +459,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0, 0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 1)), intIterable(1, 1), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0, 0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(1), eq(new TimeWindow(0, 1)), anyInternalWindowContext(), intIterable(1, 1), WindowOperatorContractTest.<Void>anyCollector());
 	}
 
 	@Test
@@ -509,16 +513,16 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 			@Override
 			public Void answer(InvocationOnMock invocation) throws Exception {
 				@SuppressWarnings("unchecked")
-				Collector<String> out = invocation.getArgumentAt(3, Collector.class);
+				Collector<String> out = invocation.getArgumentAt(4, Collector.class);
 				out.collect("Hallo");
 				out.collect("Ciao");
 				return null;
 			}
-		}).when(mockWindowFunction).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+		}).when(mockWindowFunction).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		assertThat(testHarness.extractOutputStreamRecords(),
 				contains(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
@@ -553,25 +557,25 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 			@Override
 			public Void answer(InvocationOnMock invocation) throws Exception {
 				@SuppressWarnings("unchecked")
-				Collector<String> out = invocation.getArgumentAt(3, Collector.class);
+				Collector<String> out = invocation.getArgumentAt(4, Collector.class);
 				out.collect("Hallo");
 				out.collect("Ciao");
 				return null;
 			}
-		}).when(mockWindowFunction).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+		}).when(mockWindowFunction).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		timeAdaptor.shouldRegisterTimerOnElement(mockTrigger, 1);
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, never()).apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<String>anyCollector());
+		verify(mockWindowFunction, never()).process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<String>anyCollector());
 		assertTrue(testHarness.extractOutputStreamRecords().isEmpty());
 
 		timeAdaptor.shouldFireOnTime(mockTrigger);
 
 		timeAdaptor.advanceTime(testHarness, 1L);
 
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<String>anyCollector());
 
 		assertThat(testHarness.extractOutputStreamRecords(),
 				contains(isStreamRecord("Hallo", 1L), isStreamRecord("Ciao", 1L)));
@@ -650,9 +654,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -693,9 +697,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -858,9 +862,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		timeAdaptor.advanceTime(testHarness, 0L);
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -919,9 +923,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		timeAdaptor.advanceTime(testHarness, 0L);
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -1050,7 +1054,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		timeAdaptor.verifyTriggerCallback(mockTrigger, never(), null, null);
 
 		verify(mockWindowFunction, never())
-				.apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
+				.process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
 
 		assertEquals(1, timeAdaptor.numTimers(testHarness)); // only gc timers left
 	}
@@ -1114,7 +1118,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		timeAdaptor.verifyTriggerCallback(mockTrigger, never(), null, null);
 
 		verify(mockWindowFunction, never())
-				.apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
+				.process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
 
 		assertEquals(1, timeAdaptor.numTimers(testHarness)); // only gc timers left
 	}
@@ -1186,7 +1190,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), null, null);
 
 		verify(mockWindowFunction, never())
-				.apply(anyInt(), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
+				.process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<List<Integer>>anyCollector());
 
 		// now we trigger the dangling timer
 		timeAdaptor.advanceTime(testHarness, 10L);
@@ -2208,7 +2212,7 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		testHarness.processElement(new StreamRecord<>(0, 0L));
 
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// clear is only called at cleanup time/GC time
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
@@ -2326,9 +2330,9 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		verify(mockTrigger, times(2)).clear(anyTimeWindow(), anyTriggerContext());
 
-		verify(mockWindowFunction, times(2)).apply(eq(0), anyTimeWindow(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(0, 2)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(0), eq(new TimeWindow(2, 4)), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(2)).process(eq(0), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 2)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(2, 4)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		// it's also called for the cleanup timers
 		verify(mockTrigger, times(4)).onEventTime(anyLong(), anyTimeWindow(), anyTriggerContext());
@@ -2339,6 +2343,96 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 		assertEquals(0, testHarness.numEventTimeTimers());
 	}
 
+	@Test
+	public void testPerWindowStateSetAndClearedOnEventTimePurge() throws Exception {
+		testPerWindowStateSetAndClearedOnPurge(new EventTimeAdaptor());
+	}
+
+	@Test
+	public void testPerWindowStateSetAndClearedOnProcessingTimePurge() throws Exception {
+		testPerWindowStateSetAndClearedOnPurge(new ProcessingTimeAdaptor());
+	}
+
+	public void testPerWindowStateSetAndClearedOnPurge(TimeDomainAdaptor timeAdaptor) throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		timeAdaptor.setIsEventTime(mockAssigner);
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+			createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockTrigger.onElement(anyInt(), anyLong(), anyTimeWindow(), anyTriggerContext()))
+			.thenReturn(TriggerResult.FIRE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+			.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				InternalWindowFunction.InternalWindowContext context = (InternalWindowFunction.InternalWindowContext)invocationOnMock.getArguments()[2];
+				context.windowState().getState(valueStateDescriptor).update("hello");
+				return null;
+			}
+		}).when(mockWindowFunction).process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				InternalWindowFunction.InternalWindowContext context = (InternalWindowFunction.InternalWindowContext)invocationOnMock.getArguments()[1];
+				context.windowState().getState(valueStateDescriptor).clear();
+				return null;
+			}
+		}).when(mockWindowFunction).clear(anyTimeWindow(), anyInternalWindowContext());
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+
+		assertEquals(2, testHarness.numKeyedStateEntries()); // window contents plus value state
+		assertEquals(1, timeAdaptor.numTimers(testHarness)); // gc timers
+
+		timeAdaptor.advanceTime(testHarness, 19 + 20); // 19 is maxTime of the window
+
+		assertEquals(0, testHarness.numKeyedStateEntries());
+		assertEquals(0, timeAdaptor.numTimers(testHarness));
+	}
+
+	@Test
+	public void testWindowStateNotAvailableToMergingWindows() throws Exception {
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockMergingAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+			createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockTrigger.onElement(anyInt(), anyLong(), anyTimeWindow(), anyTriggerContext()))
+			.thenReturn(TriggerResult.FIRE);
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+			.thenReturn(Arrays.asList(new TimeWindow(0, 20)));
+
+		doAnswer(new Answer<Object>() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				InternalWindowFunction.InternalWindowContext context = (InternalWindowFunction.InternalWindowContext)invocationOnMock.getArguments()[2];
+				context.windowState().getState(valueStateDescriptor).update("hello");
+				return null;
+			}
+		}).when(mockWindowFunction).process(anyInt(), anyTimeWindow(), anyInternalWindowContext(), anyIntIterable(), WindowOperatorContractTest.<Void>anyCollector());
+
+		expectedException.expect(UnsupportedOperationException.class);
+		expectedException.expectMessage("Per-window state is not allowed when using merging windows.");
+		testHarness.processElement(new StreamRecord<>(0, 0L));
+	}
+
 	protected abstract <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
 			WindowAssigner<Integer, W> assigner,
 			Trigger<Integer, W> trigger,

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
index 163117b..2f0e48e 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
@@ -22,6 +22,7 @@ import java.io.Serializable
 
 import org.apache.flink.annotation.PublicEvolving
 import org.apache.flink.api.common.functions.Function
+import org.apache.flink.api.common.state.KeyedStateStore
 import org.apache.flink.streaming.api.windowing.windows.Window
 import org.apache.flink.util.Collector
 
@@ -47,6 +48,15 @@ abstract class ProcessAllWindowFunction[IN, OUT, W <: Window] extends Function w
   def process(context: Context, elements: Iterable[IN], out: Collector[OUT])
 
   /**
+    * Deletes any state in the [[Context]] when the Window is purged.
+    *
+    * @param context The context to which the window is being evaluated
+    * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+    */
+  @throws[Exception]
+  def clear(context: Context) {}
+
+  /**
     * The context holding window metadata
     */
   abstract class Context {
@@ -54,6 +64,16 @@ abstract class ProcessAllWindowFunction[IN, OUT, W <: Window] extends Function w
       * @return The window that is being evaluated.
       */
     def window: W
+
+    /**
+      * State accessor for per-key and per-window state.
+      */
+    def windowState: KeyedStateStore
+
+    /**
+      * State accessor for per-key global state.
+      */
+    def globalState: KeyedStateStore
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessWindowFunction.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessWindowFunction.scala
index 79f3918..bdf6ae6 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessWindowFunction.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessWindowFunction.scala
@@ -22,6 +22,7 @@ import java.io.Serializable
 
 import org.apache.flink.annotation.PublicEvolving
 import org.apache.flink.api.common.functions.Function
+import org.apache.flink.api.common.state.KeyedStateStore
 import org.apache.flink.streaming.api.windowing.windows.Window
 import org.apache.flink.util.Collector
 
@@ -49,6 +50,15 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function
   def process(key: KEY, context: Context, elements: Iterable[IN], out: Collector[OUT])
 
   /**
+    * Deletes any state in the [[Context]] when the Window is purged.
+    *
+    * @param context The context to which the window is being evaluated
+    * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
+    */
+  @throws[Exception]
+  def clear(context: Context) {}
+
+  /**
     * The context holding window metadata
     */
   abstract class Context {
@@ -56,6 +66,16 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function
       * @return The window that is being evaluated.
       */
     def window: W
+
+    /**
+      * State accessor for per-key and per-window state.
+      */
+    def windowState: KeyedStateStore
+
+    /**
+      * State accessor for per-key global state.
+      */
+    def globalState: KeyedStateStore
   }
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fad201bf/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
index a4fec64..fac5958 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
@@ -52,10 +52,25 @@ final class ScalaProcessWindowFunctionWrapper[IN, OUT, KEY, W <: Window](
       out: Collector[OUT]): Unit = {
     val ctx = new func.Context {
       override def window = context.window
+
+      override def windowState = context.windowState()
+
+      override def globalState = context.globalState()
     }
     func.process(key, ctx, elements.asScala, out)
   }
 
+  override def clear(context: JProcessWindowFunction[IN, OUT, KEY, W]#Context): Unit = {
+    val ctx = new func.Context {
+      override def window = context.window
+
+      override def windowState = context.windowState()
+
+      override def globalState = context.globalState()
+    }
+    func.clear(ctx)
+  }
+
   override def setRuntimeContext(t: RuntimeContext): Unit = {
     super.setRuntimeContext(t)
     func match {
@@ -99,10 +114,26 @@ final class ScalaProcessAllWindowFunctionWrapper[IN, OUT, W <: Window](
       out: Collector[OUT]): Unit = {
     val ctx = new func.Context {
       override def window = context.window
+
+      override def windowState = context.windowState()
+
+      override def globalState = context.globalState()
     }
     func.process(ctx, elements.asScala, out)
   }
 
+  override def clear(context: JProcessAllWindowFunction[IN, OUT, W]#Context): Unit = {
+    val ctx = new func.Context {
+      override def window = context.window
+
+      override def windowState = context.windowState()
+
+      override def globalState = context.globalState()
+    }
+    func.clear(ctx)
+  }
+
+
   override def setRuntimeContext(t: RuntimeContext): Unit = {
     super.setRuntimeContext(t)
     func match {


[15/50] [abbrv] flink git commit: [FLINK-6018] Add tests for KryoSerializer restore with registered types

Posted by fh...@apache.org.
[FLINK-6018] Add tests for KryoSerializer restore with registered types

This commit also renames isCompatibleWith() to canRestoreFrom() to make
the method asymetric because in the case of KryoSerializer we
can restore from state that was stored using no registed
types/serializers while the other way around is not possible.

This closes #3534.
This closes #3603.


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

Branch: refs/heads/table-retraction
Commit: 09164cf2388888bc2f92f0ca63bb1f15283e895c
Parents: 68289b1
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Mar 16 15:17:05 2017 +0100
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Fri Mar 24 12:34:03 2017 +0800

----------------------------------------------------------------------
 .../state/RocksDBKeyedStateBackend.java         |    2 +-
 .../api/common/typeutils/TypeSerializer.java    |    2 +-
 .../typeutils/runtime/kryo/KryoSerializer.java  |   16 +
 .../AbstractKeyedCEPPatternOperator.java        |    2 +-
 .../state/AbstractKeyedStateBackend.java        |    4 +-
 .../state/DefaultOperatorStateBackend.java      |    6 +-
 .../state/RegisteredBackendStateMetaInfo.java   |    6 +-
 .../state/heap/HeapKeyedStateBackend.java       |    2 +-
 .../runtime/state/StateBackendTestBase.java     | 1346 ++++++++++++------
 9 files changed, 967 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 5b72e03..2ce527f 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -821,7 +821,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				descriptor.getSerializer());
 
 		if (stateInfo != null) {
-			if (newMetaInfo.isCompatibleWith(stateInfo.f1)) {
+			if (newMetaInfo.canRestoreFrom(stateInfo.f1)) {
 				stateInfo.f1 = newMetaInfo;
 				return stateInfo.f0;
 			} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
index ac7fbc8..6edaec6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
@@ -161,7 +161,7 @@ public abstract class TypeSerializer<T> implements Serializable {
 
 	public abstract int hashCode();
 
-	public boolean isCompatibleWith(TypeSerializer<?> other) {
+	public boolean canRestoreFrom(TypeSerializer<?> other) {
 		return equals(other);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index 44c952a..cba0c84 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -383,4 +383,20 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 		checkKryoInitialized();
 		return this.kryo;
 	}
+
+	@Override
+	public boolean canRestoreFrom(TypeSerializer<?> other) {
+		if (other instanceof KryoSerializer) {
+			KryoSerializer<?> otherKryo = (KryoSerializer<?>) other;
+
+			// we cannot include the Serializers here because they don't implement the equals method
+			return other.canEqual(this) &&
+					type == otherKryo.type &&
+					(registeredTypes.equals(otherKryo.registeredTypes) || otherKryo.registeredTypes.isEmpty()) &&
+					(registeredTypesWithSerializerClasses.equals(otherKryo.registeredTypesWithSerializerClasses) || otherKryo.registeredTypesWithSerializerClasses.isEmpty()) &&
+					(defaultSerializerClasses.equals(otherKryo.defaultSerializerClasses) || otherKryo.defaultSerializerClasses.isEmpty());
+		} else {
+			return false;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index b6d57cd..3e18660 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -404,7 +404,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 		}
 
 		@Override
-		public boolean isCompatibleWith(TypeSerializer<?> other) {
+		public boolean canRestoreFrom(TypeSerializer<?> other) {
 			return equals(other) || other instanceof AbstractKeyedCEPPatternOperator.PriorityQueueSerializer;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index 1f2f4a2..e6e7b23 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -277,7 +277,7 @@ public abstract class AbstractKeyedStateBackend<K>
 		}
 
 		if (!stateDescriptor.isSerializerInitialized()) {
-			throw new IllegalStateException("The serializer of the descriptor has not been initialized!"); 
+			stateDescriptor.initializeSerializerUnlessSet(executionConfig);
 		}
 
 		InternalKvState<?> existing = keyValueStatesByName.get(stateDescriptor.getName());
@@ -355,8 +355,6 @@ public abstract class AbstractKeyedStateBackend<K>
 
 		checkNotNull(namespace, "Namespace");
 
-		stateDescriptor.initializeSerializerUnlessSet(executionConfig);
-
 		if (lastName != null && lastName.equals(stateDescriptor.getName())) {
 			lastState.setCurrentNamespace(namespace);
 			return (S) lastState;

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
index 71cccae..ca7cb48 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
@@ -134,8 +134,8 @@ public class DefaultOperatorStateBackend implements OperatorStateBackend {
 					"Incompatible assignment mode. Provided: " + mode + ", expected: " +
 							partitionableListState.getAssignmentMode());
 			Preconditions.checkState(
-					partitionableListState.getPartitionStateSerializer().
-							isCompatibleWith(stateDescriptor.getElementSerializer()),
+					stateDescriptor.getElementSerializer().
+							canRestoreFrom(partitionableListState.getPartitionStateSerializer()),
 					"Incompatible type serializers. Provided: " + stateDescriptor.getElementSerializer() +
 							", found: " + partitionableListState.getPartitionStateSerializer());
 		}
@@ -258,7 +258,7 @@ public class DefaultOperatorStateBackend implements OperatorStateBackend {
 
 						registeredStates.put(listState.getName(), listState);
 					} else {
-						Preconditions.checkState(listState.getPartitionStateSerializer().isCompatibleWith(
+						Preconditions.checkState(listState.getPartitionStateSerializer().canRestoreFrom(
 								stateMetaInfo.getStateSerializer()), "Incompatible state serializers found: " +
 								listState.getPartitionStateSerializer() + " is not compatible with " +
 								stateMetaInfo.getStateSerializer());

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
index 80bdacd..0d4b3c8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
@@ -74,7 +74,7 @@ public class RegisteredBackendStateMetaInfo<N, S> {
 		return stateSerializer;
 	}
 
-	public boolean isCompatibleWith(RegisteredBackendStateMetaInfo<?, ?> other) {
+	public boolean canRestoreFrom(RegisteredBackendStateMetaInfo<?, ?> other) {
 
 		if (this == other) {
 			return true;
@@ -94,8 +94,8 @@ public class RegisteredBackendStateMetaInfo<N, S> {
 			return false;
 		}
 
-		return (stateSerializer.isCompatibleWith(other.stateSerializer)) &&
-				(namespaceSerializer.isCompatibleWith(other.namespaceSerializer)
+		return (stateSerializer.canRestoreFrom(other.stateSerializer)) &&
+				(namespaceSerializer.canRestoreFrom(other.namespaceSerializer)
 						// we also check if there is just a migration proxy that should be replaced by any real serializer
 						|| other.namespaceSerializer instanceof MigrationNamespaceSerializerProxy);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index f3e4ec6..46ec5c2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -141,7 +141,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			stateTable = newStateTable(newMetaInfo);
 			stateTables.put(stateName, stateTable);
 		} else {
-			if (!newMetaInfo.isCompatibleWith(stateTable.getMetaInfo())) {
+			if (!newMetaInfo.canRestoreFrom(stateTable.getMetaInfo())) {
 				throw new RuntimeException("Trying to access state using incompatible meta info, was " +
 						stateTable.getMetaInfo() + " trying access with " + newMetaInfo);
 			}


[32/50] [abbrv] flink git commit: [FLINK-6197] [cep] Add support for iterative conditions.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
index 68b0419..e9aa7d2 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
@@ -18,9 +18,11 @@
 
 package org.apache.flink.cep.pattern;
 
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
+import org.apache.flink.cep.pattern.conditions.OrCondition;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.cep.pattern.conditions.SubtypeCondition;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
@@ -66,14 +68,14 @@ public class PatternTest extends TestLogger {
 
 	@Test
 	public void testStrictContiguityWithCondition() {
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").next("next").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").next("next").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -7657256242101104925L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("foobar");
 			}
-		}).next("end").where(new FilterFunction<Event>() {
+		}).next("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -7597452389191504189L;
 
 			@Override
@@ -89,9 +91,9 @@ public class PatternTest extends TestLogger {
 		assertNotNull(previous2 = previous.getPrevious());
 		assertNull(previous2.getPrevious());
 
-		assertNotNull(pattern.getFilterFunction());
-		assertNotNull(previous.getFilterFunction());
-		assertNull(previous2.getFilterFunction());
+		assertNotNull(pattern.getCondition());
+		assertNotNull(previous.getCondition());
+		assertNull(previous2.getCondition());
 
 		assertEquals(pattern.getName(), "end");
 		assertEquals(previous.getName(), "next");
@@ -109,8 +111,8 @@ public class PatternTest extends TestLogger {
 		assertNotNull(previous2 = previous.getPrevious());
 		assertNull(previous2.getPrevious());
 
-		assertNotNull(previous.getFilterFunction());
-		assertTrue(previous.getFilterFunction() instanceof SubtypeFilterFunction);
+		assertNotNull(previous.getCondition());
+		assertTrue(previous.getCondition() instanceof SubtypeCondition);
 
 		assertEquals(pattern.getName(), "end");
 		assertEquals(previous.getName(), "subevent");
@@ -119,7 +121,7 @@ public class PatternTest extends TestLogger {
 
 	@Test
 	public void testPatternWithSubtypingAndFilter() {
-		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").next("subevent").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
+		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").next("subevent").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
 			private static final long serialVersionUID = -4118591291880230304L;
 
 			@Override
@@ -136,7 +138,7 @@ public class PatternTest extends TestLogger {
 		assertNull(previous2.getPrevious());
 
 		assertTrue(pattern instanceof FollowedByPattern);
-		assertNotNull(previous.getFilterFunction());
+		assertNotNull(previous.getCondition());
 
 		assertEquals(pattern.getName(), "end");
 		assertEquals(previous.getName(), "subevent");
@@ -145,21 +147,21 @@ public class PatternTest extends TestLogger {
 
 	@Test
 	public void testPatternWithOrFilter() {
-		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 3518061453394250543L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return false;
 			}
-		}).or(new FilterFunction<Event>() {
+		}).or(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 947463545810023841L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return false;
 			}
-		}).next("or").or(new FilterFunction<Event>() {
+		}).next("or").or(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -2775487887505922250L;
 
 			@Override
@@ -176,8 +178,8 @@ public class PatternTest extends TestLogger {
 		assertNull(previous2.getPrevious());
 
 		assertTrue(pattern instanceof FollowedByPattern);
-		assertFalse(previous.getFilterFunction() instanceof OrFilterFunction);
-		assertTrue(previous2.getFilterFunction() instanceof OrFilterFunction);
+		assertFalse(previous.getCondition() instanceof OrCondition);
+		assertTrue(previous2.getCondition() instanceof OrCondition);
 
 		assertEquals(pattern.getName(), "end");
 		assertEquals(previous.getName(), "or");
@@ -187,7 +189,9 @@ public class PatternTest extends TestLogger {
 	@Test(expected = MalformedPatternException.class)
 	public void testPatternCanHaveQuantifierSpecifiedOnce1() throws Exception {
 
-		Pattern.begin("start").where(new FilterFunction<Object>() {
+		Pattern.begin("start").where(new SimpleCondition<Object>() {
+			private static final long serialVersionUID = 8876425689668531458L;
+
 			@Override
 			public boolean filter(Object value) throws Exception {
 				return true;
@@ -198,7 +202,9 @@ public class PatternTest extends TestLogger {
 	@Test(expected = MalformedPatternException.class)
 	public void testPatternCanHaveQuantifierSpecifiedOnce2() throws Exception {
 
-		Pattern.begin("start").where(new FilterFunction<Object>() {
+		Pattern.begin("start").where(new SimpleCondition<Object>() {
+			private static final long serialVersionUID = 8311890695733430258L;
+
 			@Override
 			public boolean filter(Object value) throws Exception {
 				return true;
@@ -209,7 +215,9 @@ public class PatternTest extends TestLogger {
 	@Test(expected = MalformedPatternException.class)
 	public void testPatternCanHaveQuantifierSpecifiedOnce3() throws Exception {
 
-		Pattern.begin("start").where(new FilterFunction<Object>() {
+		Pattern.begin("start").where(new SimpleCondition<Object>() {
+			private static final long serialVersionUID = 8093713196099078214L;
+
 			@Override
 			public boolean filter(Object value) throws Exception {
 				return true;
@@ -220,7 +228,9 @@ public class PatternTest extends TestLogger {
 	@Test(expected = MalformedPatternException.class)
 	public void testPatternCanHaveQuantifierSpecifiedOnce4() throws Exception {
 
-		Pattern.begin("start").where(new FilterFunction<Object>() {
+		Pattern.begin("start").where(new SimpleCondition<Object>() {
+			private static final long serialVersionUID = -2995187062849334113L;
+
 			@Override
 			public boolean filter(Object value) throws Exception {
 				return true;
@@ -231,7 +241,9 @@ public class PatternTest extends TestLogger {
 	@Test(expected = MalformedPatternException.class)
 	public void testPatternCanHaveQuantifierSpecifiedOnce5() throws Exception {
 
-		Pattern.begin("start").where(new FilterFunction<Object>() {
+		Pattern.begin("start").where(new SimpleCondition<Object>() {
+			private static final long serialVersionUID = -2205071036073867531L;
+
 			@Override
 			public boolean filter(Object value) throws Exception {
 				return true;


[10/50] [abbrv] flink git commit: [FLINK-6129] [metrics] Stop query actor of MetricRegistry

Posted by fh...@apache.org.
[FLINK-6129] [metrics] Stop query actor of MetricRegistry

This PR properly shuts down the query actor of the MetricRegistry upon shut down.

Add locking to the MetricRegistry

This closes #3573.


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

Branch: refs/heads/table-retraction
Commit: 8319a457d9adee310ef64905709c03ca2f2afd61
Parents: 3e860b4
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Mar 20 14:55:30 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Mar 23 15:13:33 2017 +0100

----------------------------------------------------------------------
 .../flink/runtime/metrics/MetricRegistry.java   | 157 +++++++++++++------
 .../runtime/metrics/MetricRegistryTest.java     |  34 ++++
 2 files changed, 142 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8319a457/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java
index d3b21fc..9f46d47 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java
@@ -20,6 +20,9 @@ package org.apache.flink.runtime.metrics;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
+import akka.actor.Kill;
+import akka.pattern.Patterns;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -34,8 +37,12 @@ import org.apache.flink.runtime.metrics.dump.MetricQueryService;
 import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
 import org.apache.flink.runtime.metrics.groups.FrontMetricGroup;
 import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -52,7 +59,9 @@ import java.util.concurrent.atomic.AtomicInteger;
  */
 public class MetricRegistry {
 	static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class);
-	
+
+	private final Object lock = new Object();
+
 	private List<MetricReporter> reporters;
 	private ScheduledExecutorService executor;
 	private ActorRef queryService;
@@ -150,10 +159,14 @@ public class MetricRegistry {
 	 * @param resourceID resource ID used to disambiguate the actor name
      */
 	public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) {
-		try {
-			queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID);
-		} catch (Exception e) {
-			LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e);
+		synchronized (lock) {
+			Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down.");
+
+			try {
+				queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID);
+			} catch (Exception e) {
+				LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e);
+			}
 		}
 	}
 
@@ -191,24 +204,51 @@ public class MetricRegistry {
 	 * @return true, if this registry was shutdown, otherwise false
 	 */
 	public boolean isShutdown() {
-		return reporters == null && executor.isShutdown();
+		synchronized (lock) {
+			return reporters == null && executor.isShutdown();
+		}
 	}
 
 	/**
 	 * Shuts down this registry and the associated {@link MetricReporter}.
 	 */
 	public void shutdown() {
-		if (reporters != null) {
-			for (MetricReporter reporter : reporters) {
+		synchronized (lock) {
+			Future<Boolean> stopFuture = null;
+			FiniteDuration stopTimeout = null;
+
+			if (queryService != null) {
+				stopTimeout = new FiniteDuration(1L, TimeUnit.SECONDS);
+				stopFuture = Patterns.gracefulStop(queryService, stopTimeout);
+			}
+
+			if (reporters != null) {
+				for (MetricReporter reporter : reporters) {
+					try {
+						reporter.close();
+					} catch (Throwable t) {
+						LOG.warn("Metrics reporter did not shut down cleanly", t);
+					}
+				}
+				reporters = null;
+			}
+			shutdownExecutor();
+
+			if (stopFuture != null) {
+				boolean stopped = false;
+
 				try {
-					reporter.close();
-				} catch (Throwable t) {
-					LOG.warn("Metrics reporter did not shut down cleanly", t);
+					stopped = Await.result(stopFuture, stopTimeout);
+				} catch (Exception e) {
+					LOG.warn("Query actor did not properly stop.", e);
+				}
+
+				if (!stopped) {
+					// the query actor did not stop in time, let's kill him
+					queryService.tell(Kill.getInstance(), ActorRef.noSender());
 				}
 			}
-			reporters = null;
 		}
-		shutdownExecutor();
 	}
 	
 	private void shutdownExecutor() {
@@ -216,7 +256,7 @@ public class MetricRegistry {
 			executor.shutdown();
 
 			try {
-				if (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+				if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) {
 					executor.shutdownNow();
 				}
 			} catch (InterruptedException e) {
@@ -241,27 +281,33 @@ public class MetricRegistry {
 	 * @param group       the group that contains the metric
 	 */
 	public void register(Metric metric, String metricName, AbstractMetricGroup group) {
-		try {
-			if (reporters != null) {
-				for (int i = 0; i < reporters.size(); i++) {
-					MetricReporter reporter = reporters.get(i);
-					if (reporter != null) {
-						FrontMetricGroup front = new FrontMetricGroup<AbstractMetricGroup<?>>(i, group);
-						reporter.notifyOfAddedMetric(metric, metricName, front);
+		synchronized (lock) {
+			if (isShutdown()) {
+				LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down.");
+			} else {
+				try {
+					if (reporters != null) {
+						for (int i = 0; i < reporters.size(); i++) {
+							MetricReporter reporter = reporters.get(i);
+							if (reporter != null) {
+								FrontMetricGroup front = new FrontMetricGroup<AbstractMetricGroup<?>>(i, group);
+								reporter.notifyOfAddedMetric(metric, metricName, front);
+							}
+						}
 					}
+					if (queryService != null) {
+						MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group);
+					}
+					if (metric instanceof View) {
+						if (viewUpdater == null) {
+							viewUpdater = new ViewUpdater(executor);
+						}
+						viewUpdater.notifyOfAddedView((View) metric);
+					}
+				} catch (Exception e) {
+					LOG.error("Error while registering metric.", e);
 				}
 			}
-			if (queryService != null) {
-				MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group);
-			}
-			if (metric instanceof View) {
-				if (viewUpdater == null) {
-					viewUpdater = new ViewUpdater(executor);
-				}
-				viewUpdater.notifyOfAddedView((View) metric);
-			}
-		} catch (Exception e) {
-			LOG.error("Error while registering metric.", e);
 		}
 	}
 
@@ -273,31 +319,44 @@ public class MetricRegistry {
 	 * @param group       the group that contains the metric
 	 */
 	public void unregister(Metric metric, String metricName, AbstractMetricGroup group) {
-		try {
-			if (reporters != null) {
-				for (int i = 0; i < reporters.size(); i++) {
-					MetricReporter reporter = reporters.get(i);
-					if (reporter != null) {
-						FrontMetricGroup front = new FrontMetricGroup<AbstractMetricGroup<?>>(i, group);
-						reporter.notifyOfRemovedMetric(metric, metricName, front);
+		synchronized (lock) {
+			if (isShutdown()) {
+				LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down.");
+			} else {
+				try {
+					if (reporters != null) {
+						for (int i = 0; i < reporters.size(); i++) {
+							MetricReporter reporter = reporters.get(i);
+							if (reporter != null) {
+								FrontMetricGroup front = new FrontMetricGroup<AbstractMetricGroup<?>>(i, group);
+								reporter.notifyOfRemovedMetric(metric, metricName, front);
+							}
+						}
 					}
+					if (queryService != null) {
+						MetricQueryService.notifyOfRemovedMetric(queryService, metric);
+					}
+					if (metric instanceof View) {
+						if (viewUpdater != null) {
+							viewUpdater.notifyOfRemovedView((View) metric);
+						}
+					}
+				} catch (Exception e) {
+					LOG.error("Error while registering metric.", e);
 				}
 			}
-			if (queryService != null) {
-				MetricQueryService.notifyOfRemovedMetric(queryService, metric);
-			}
-			if (metric instanceof View) {
-				if (viewUpdater != null) {
-					viewUpdater.notifyOfRemovedView((View) metric);
-				}
-			}
-		} catch (Exception e) {
-			LOG.error("Error while registering metric.", e);
 		}
 	}
 
 	// ------------------------------------------------------------------------
 
+	@VisibleForTesting
+	public ActorRef getQueryService() {
+		return queryService;
+	}
+
+	// ------------------------------------------------------------------------
+
 	/**
 	 * This task is explicitly a static class, so that it does not hold any references to the enclosing
 	 * MetricsRegistry instance.

http://git-wip-us.apache.org/repos/asf/flink/blob/8319a457/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java
index ab4e7a4..fe29ccb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.metrics;
 
+import akka.actor.ActorNotFound;
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.metrics.Counter;
@@ -26,6 +29,7 @@ import org.apache.flink.metrics.MetricConfig;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.metrics.reporter.MetricReporter;
 import org.apache.flink.metrics.reporter.Scheduled;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.metrics.scope.ScopeFormats;
 import org.apache.flink.runtime.metrics.util.TestReporter;
@@ -33,11 +37,15 @@ import org.apache.flink.runtime.metrics.util.TestReporter;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class MetricRegistryTest extends TestLogger {
 
@@ -348,6 +356,32 @@ public class MetricRegistryTest extends TestLogger {
 		assertEquals(4, TestReporter8.numCorrectDelimitersForUnregister);
 	}
 
+	/**
+	 * Tests that the query actor will be stopped when the MetricRegistry is shut down.
+	 */
+	@Test
+	public void testQueryActorShutdown() throws Exception {
+		final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS);
+
+		MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration());
+
+		final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+
+		registry.startQueryService(actorSystem, null);
+
+		ActorRef queryServiceActor = registry.getQueryService();
+
+		registry.shutdown();
+
+		try {
+			Await.result(actorSystem.actorSelection(queryServiceActor.path()).resolveOne(timeout), timeout);
+
+			fail("The query actor should be terminated resulting in a ActorNotFound exception.");
+		} catch (ActorNotFound e) {
+			// we expect the query actor to be shut down
+		}
+	}
+
 	public static class TestReporter8 extends TestReporter {
 		char expectedDelimiter;
 		public static int numCorrectDelimitersForRegister = 0;


[46/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Remove redundant method 'ExecutionVertex.getSimpleName()'

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Remove redundant method 'ExecutionVertex.getSimpleName()'

Replace the method via identical method 'getTaskNameWithSubtaskIndex'.


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

Branch: refs/heads/table-retraction
Commit: ca681101fa7c813345dc3125a3ec7af22563ab00
Parents: 719d0cf
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 29 22:32:53 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 22:32:53 2017 +0200

----------------------------------------------------------------------
 .../checkpoint/CheckpointCoordinator.java       |  4 ++--
 .../flink/runtime/executiongraph/Execution.java |  6 ++---
 .../runtime/executiongraph/ExecutionVertex.java | 24 ++++++++------------
 .../runtime/jobmanager/scheduler/Scheduler.java |  2 +-
 .../ExecutionGraphDeploymentTest.java           |  2 +-
 .../scheduler/SchedulerTestUtils.java           |  2 +-
 6 files changed, 18 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index cc60837..7087540 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -435,7 +435,7 @@ public class CheckpointCoordinator {
 				executions[i] = ee;
 			} else {
 				LOG.info("Checkpoint triggering task {} is not being executed at the moment. Aborting checkpoint.",
-						tasksToTrigger[i].getSimpleName());
+						tasksToTrigger[i].getTaskNameWithSubtaskIndex());
 				return new CheckpointTriggerResult(CheckpointDeclineReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
 			}
 		}
@@ -450,7 +450,7 @@ public class CheckpointCoordinator {
 				ackTasks.put(ee.getAttemptId(), ev);
 			} else {
 				LOG.info("Checkpoint acknowledging task {} is not being executed at the moment. Aborting checkpoint.",
-						ev.getSimpleName());
+						ev.getTaskNameWithSubtaskIndex());
 				return new CheckpointTriggerResult(CheckpointDeclineReason.NOT_ALL_REQUIRED_TASKS_RUNNING);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 1a3ef11..729e161 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -357,7 +357,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 			}
 
 			if (LOG.isInfoEnabled()) {
-				LOG.info(String.format("Deploying %s (attempt #%d) to %s", vertex.getSimpleName(),
+				LOG.info(String.format("Deploying %s (attempt #%d) to %s", vertex.getTaskNameWithSubtaskIndex(),
 						attemptNumber, getAssignedResourceLocation().getHostname()));
 			}
 
@@ -1071,7 +1071,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 	}
 
 	public String getVertexWithAttempt() {
-		return vertex.getSimpleName() + " - execution #" + attemptNumber;
+		return vertex.getTaskNameWithSubtaskIndex() + " - execution #" + attemptNumber;
 	}
 
 	// ------------------------------------------------------------------------
@@ -1126,7 +1126,7 @@ public class Execution implements AccessExecution, Archiveable<ArchivedExecution
 	
 	@Override
 	public String toString() {
-		return String.format("Attempt #%d (%s) @ %s - [%s]", attemptNumber, vertex.getSimpleName(),
+		return String.format("Attempt #%d (%s) @ %s - [%s]", attemptNumber, vertex.getTaskNameWithSubtaskIndex(),
 				(assignedResource == null ? "(unassigned)" : assignedResource.toString()), state);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index c7829fa..90820e9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -188,6 +188,14 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 		return this.jobVertex.getJobVertex().getName();
 	}
 
+	/**
+	 * Creates a simple name representation in the style 'taskname (x/y)', where
+	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
+	 * subtask index as returned by {@link #getParallelSubtaskIndex()}{@code + 1}, and 'y' is the total
+	 * number of tasks, as returned by {@link #getTotalNumberOfParallelSubtasks()}.
+	 *
+	 * @return A simple name representation in the form 'myTask (2/7)'
+	 */
 	@Override
 	public String getTaskNameWithSubtaskIndex() {
 		return this.taskNameWithSubtask;
@@ -503,7 +511,7 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 
 	public void resetForNewExecution() {
 
-		LOG.debug("Resetting execution vertex {} for new execution.", getSimpleName());
+		LOG.debug("Resetting execution vertex {} for new execution.", getTaskNameWithSubtaskIndex());
 
 		synchronized (priorExecutions) {
 			Execution execution = currentExecution;
@@ -722,21 +730,9 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable<Archi
 	//  Utilities
 	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * Creates a simple name representation in the style 'taskname (x/y)', where
-	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
-	 * subtask index as returned by {@link #getParallelSubtaskIndex()}{@code + 1}, and 'y' is the total
-	 * number of tasks, as returned by {@link #getTotalNumberOfParallelSubtasks()}.
-	 *
-	 * @return A simple name representation.
-	 */
-	public String getSimpleName() {
-		return taskNameWithSubtask;
-	}
-
 	@Override
 	public String toString() {
-		return getSimpleName();
+		return getTaskNameWithSubtaskIndex();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index 58dac3e..af72d7c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -570,7 +570,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener, Sl
 								queued.getFuture().complete(newSlot);
 							}
 							catch (Throwable t) {
-								LOG.error("Error calling allocation future for task " + vertex.getSimpleName(), t);
+								LOG.error("Error calling allocation future for task " + vertex.getTaskNameWithSubtaskIndex(), t);
 								task.getTaskToExecute().fail(t);
 							}
 						}

http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index 7f5811a..8d91b84 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -359,7 +359,7 @@ public class ExecutionGraphDeploymentTest {
 			Collections.sort(execList, new Comparator<Execution>() {
 				@Override
 				public int compare(Execution o1, Execution o2) {
-					return o1.getVertex().getSimpleName().compareTo(o2.getVertex().getSimpleName());
+					return o1.getVertex().getTaskNameWithSubtaskIndex().compareTo(o2.getVertex().getTaskNameWithSubtaskIndex());
 				}
 			});
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ca681101/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
index 9e692ff..4312b0f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
@@ -126,7 +126,7 @@ public class SchedulerTestUtils {
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
 		when(vertex.getMaxParallelism()).thenReturn(numTasks);
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
-		when(vertex.getSimpleName()).thenReturn("TEST-VERTEX");
+		when(vertex.getTaskNameWithSubtaskIndex()).thenReturn("TEST-VERTEX");
 		
 		Execution execution = mock(Execution.class);
 		when(execution.getVertex()).thenReturn(vertex);


[25/50] [abbrv] flink git commit: [hotfix] [table] Disable event-time OVER RANGE UNBOUNDED PRECEDING window.

Posted by fh...@apache.org.
[hotfix] [table] Disable event-time OVER RANGE UNBOUNDED PRECEDING window.


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

Branch: refs/heads/table-retraction
Commit: cac9fa0288e85c5d8315c71d4a39e1e926102e07
Parents: fe2c61a
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Mar 24 21:22:11 2017 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 21:23:25 2017 +0100

----------------------------------------------------------------------
 .../datastream/DataStreamOverAggregate.scala    | 21 ++++++----
 .../table/api/scala/stream/sql/SqlITCase.scala  | 40 ++++++++++----------
 2 files changed, 33 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cac9fa02/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 3dd7ee2..01e5a9a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -113,23 +113,28 @@ class DataStreamOverAggregate(
           if (overWindow.isRows) {
             // ROWS clause bounded OVER window
             throw new TableException(
-              "ROWS clause bounded proc-time OVER window no supported yet.")
+              "processing-time OVER ROWS PRECEDING window is not supported yet.")
           } else {
             // RANGE clause bounded OVER window
             throw new TableException(
-              "RANGE clause bounded proc-time OVER window no supported yet.")
+              "processing-time OVER RANGE PRECEDING window is not supported yet.")
           }
         } else {
           throw new TableException(
-            "OVER window only support ProcessingTime UNBOUNDED PRECEDING and CURRENT ROW " +
-                "condition.")
+            "processing-time OVER RANGE FOLLOWING window is not supported yet.")
         }
       case _: RowTimeType =>
         // row-time OVER window
         if (overWindow.lowerBound.isPreceding &&
               overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
-          // unbounded preceding OVER window
-          createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
+          if (overWindow.isRows) {
+            // unbounded preceding OVER ROWS window
+            createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
+          } else {
+            // unbounded preceding OVER RANGE window
+            throw new TableException(
+              "row-time OVER RANGE UNBOUNDED PRECEDING window is not supported yet.")
+          }
         } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
           // bounded OVER window
           if (overWindow.isRows) {
@@ -138,11 +143,11 @@ class DataStreamOverAggregate(
           } else {
             // RANGE clause bounded OVER window
             throw new TableException(
-              "RANGE clause bounded row-time OVER window no supported yet.")
+              "row-time OVER RANGE PRECEDING window is not supported yet.")
           }
         } else {
           throw new TableException(
-            "row-time OVER window only support CURRENT ROW condition.")
+            "row-time OVER RANGE FOLLOWING window is not supported yet.")
         }
       case _ =>
         throw new TableException(s"Unsupported time type {$timeType}")

http://git-wip-us.apache.org/repos/asf/flink/blob/cac9fa02/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 34a68b2..80ff42a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -448,15 +448,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "count(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "avg(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "max(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "min(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -526,15 +526,15 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val sqlQuery = "SELECT a, b, c, " +
       "SUM(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "count(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "avg(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "max(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row), " +
       "min(b) over (" +
-      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "partition by a order by rowtime() rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -596,11 +596,11 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
-      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "SUM(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime() rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(
@@ -651,11 +651,11 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.setParallelism(1)
 
     val sqlQuery = "SELECT a, b, c, " +
-      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
-      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "SUM(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime() rows between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime() rows between unbounded preceding and current row) " +
       "from T1"
 
     val data = Seq(


[35/50] [abbrv] flink git commit: comments

Posted by fh...@apache.org.
comments


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

Branch: refs/heads/table-retraction
Commit: ad21a441434b9ac5886b664871553bf57885e984
Parents: 7fbdc10
Author: kl0u <kk...@gmail.com>
Authored: Tue Mar 28 10:46:49 2017 +0200
Committer: kl0u <kk...@gmail.com>
Committed: Tue Mar 28 11:08:04 2017 +0200

----------------------------------------------------------------------
 docs/dev/libs/cep.md                                 | 15 +++------------
 .../org/apache/flink/cep/scala/pattern/Pattern.scala | 15 +++++++++++++++
 .../src/main/java/org/apache/flink/cep/nfa/NFA.java  |  6 +++---
 .../cep/pattern/conditions/IterativeCondition.java   |  2 +-
 4 files changed, 22 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ad21a441/docs/dev/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index 9d4ca91..932ba30 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -146,11 +146,10 @@ start.where(new IterativeCondition<SubEvent>() {
             return false;
         }
         
-        double sum = 0.0;
+        double sum = value.getPrice();
         for (Event event : ctx.getEventsForPattern("middle")) {
             sum += event.getPrice();
         }
-        sum += value.getPrice();
         return Double.compare(sum, 5.0) < 0;
     }
 });
@@ -161,16 +160,8 @@ start.where(new IterativeCondition<SubEvent>() {
 {% highlight scala %}
 start.where(
     (value, ctx) => {
-        var res = value.getName.startsWith("foo")
-        if (res) {
-            var sum = 0.0
-            for (e: Event <- ctx.getEventsForPattern("middle")) {
-                sum += e.getPrice
-            }
-            sum += value.getPrice
-            res = res && sum < 5.0
-        }
-        res
+        lazy val sum = ctx.getEventsForPattern("middle").asScala.map(_.getPrice).sum
+        value.getName.startsWith("foo") && sum + value.getPrice < 5.0
     }
 )
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/flink/blob/ad21a441/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
index a1db460..07dfc5a 100644
--- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
+++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
@@ -145,6 +145,21 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
   }
 
   /**
+    * Specifies a filter condition which is ORed with an existing filter function.
+    *
+    * @param filterFun Or filter function
+    * @return The same pattern operator where the new filter condition is set
+    */
+  def or(filterFun: (F, Context[F]) => Boolean): Pattern[T, F] = {
+    val filter = new IterativeCondition[F] {
+      val cleanFilter = cep.scala.cleanClosure(filterFun)
+
+      override def filter(value: F, ctx: Context[F]): Boolean = cleanFilter(value, ctx)
+    }
+    or(filter)
+  }
+
+  /**
     * Specifies a filter condition which has to be fulfilled by an event in order to be matched.
     *
     * @param filterFun Filter condition

http://git-wip-us.apache.org/repos/asf/flink/blob/ad21a441/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index cddc1ed..98c1fc9 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -92,7 +92,7 @@ public class NFA<T> implements Serializable {
 	private final NonDuplicatingTypeSerializer<T> nonDuplicatingTypeSerializer;
 
 	/**
-	 * 	Used only for backward compatibility. Buffer used to store the matched events.
+	 * Used only for backwards compatibility. Buffer used to store the matched events.
 	 */
 	private final SharedBuffer<State<T>, T> sharedBuffer = null;
 
@@ -575,7 +575,7 @@ public class NFA<T> implements Serializable {
 				computationState.getVersion());
 
 		// for a given computation state, we cannot have more than one matching patterns.
-		Preconditions.checkArgument(paths.size() <= 1);
+		Preconditions.checkState(paths.size() <= 1);
 
 		TypeSerializer<T> serializer = nonDuplicatingTypeSerializer.getTypeSerializer();
 
@@ -609,7 +609,7 @@ public class NFA<T> implements Serializable {
 			computationState.getVersion());
 
 		// for a given computation state, we cannot have more than one matching patterns.
-		Preconditions.checkArgument(paths.size() <= 1);
+		Preconditions.checkState(paths.size() <= 1);
 
 		List<Map<String, T>> result = new ArrayList<>();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ad21a441/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
index f225e01..016cdef 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
@@ -88,7 +88,7 @@ public abstract class IterativeCondition<T> implements Function, Serializable {
 
 		/**
 		 * @return An {@link Iterable} over the already accepted elements
-		 * for a given pattern. Elements are iterated in the order the were
+		 * for a given pattern. Elements are iterated in the order they were
 		 * inserted in the pattern.
 		 *
 		 * @param name The name of the pattern.


[09/50] [abbrv] flink git commit: [hotfix][scripts] Fix zookeeper quorum regex

Posted by fh...@apache.org.
[hotfix][scripts] Fix zookeeper quorum regex


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

Branch: refs/heads/table-retraction
Commit: 3e860b40761b27904edecb981aa7a29bc1e89aec
Parents: 83b99f8
Author: Robert Metzger <rm...@apache.org>
Authored: Thu Mar 23 15:11:57 2017 +0100
Committer: Robert Metzger <rm...@apache.org>
Committed: Thu Mar 23 15:11:57 2017 +0100

----------------------------------------------------------------------
 flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3e860b40/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh b/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh
index 67d5349..05a5208 100755
--- a/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh
+++ b/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh
@@ -35,7 +35,7 @@ while read server ; do
     server=$(echo -e "${server}" | sed -e 's/^[[:space:]]*//' -e 's/[[:space:]]*$//') # trim
 
     # match server.id=address[:port[:port]]
-    if [[ $server =~ ^server\.([0-9])+[[:space:]]*\=([^: \#]+) ]]; then
+    if [[ $server =~ ^server\.([0-9]+)[[:space:]]*\=([^: \#]+) ]]; then
         id=${BASH_REMATCH[1]}
         address=${BASH_REMATCH[2]}
 


[24/50] [abbrv] flink git commit: [FLINK-6089] [table] Add decoration phase for stream queries to rewrite plans after the cost-based optimization.

Posted by fh...@apache.org.
[FLINK-6089] [table] Add decoration phase for stream queries to rewrite plans after the cost-based optimization.

This closes #3564.


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

Branch: refs/heads/table-retraction
Commit: 6949c8c79c41344023df08dde2936f06daa00e0d
Parents: f97deaa
Author: hequn.chq <he...@alibaba-inc.com>
Authored: Thu Mar 16 11:11:17 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 20:19:17 2017 +0100

----------------------------------------------------------------------
 .../table/api/StreamTableEnvironment.scala      | 38 ++++++++-
 .../flink/table/calcite/CalciteConfig.scala     | 89 +++++++++++++++++---
 .../flink/table/plan/rules/FlinkRuleSets.scala  |  9 +-
 .../flink/table/CalciteConfigBuilderTest.scala  | 69 +++++++++++++++
 4 files changed, 188 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6949c8c7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index d927c3a..225a675 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -25,7 +25,7 @@ import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql2rel.RelDecorrelator
-import org.apache.calcite.tools.RuleSet
+import org.apache.calcite.tools.{RuleSet, RuleSets}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
@@ -39,6 +39,8 @@ import org.apache.flink.table.sinks.{StreamTableSink, TableSink}
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 import org.apache.flink.types.Row
 
+import _root_.scala.collection.JavaConverters._
+
 /**
   * The base class for stream TableEnvironments.
   *
@@ -211,6 +213,26 @@ abstract class StreamTableEnvironment(
   }
 
   /**
+    * Returns the decoration rule set for this environment
+    * including a custom RuleSet configuration.
+    */
+  protected def getDecoRuleSet: RuleSet = {
+    val calciteConfig = config.getCalciteConfig
+    calciteConfig.getDecoRuleSet match {
+
+      case None =>
+        getBuiltInDecoRuleSet
+
+      case Some(ruleSet) =>
+        if (calciteConfig.replacesDecoRuleSet) {
+          ruleSet
+        } else {
+          RuleSets.ofList((getBuiltInDecoRuleSet.asScala ++ ruleSet.asScala).asJava)
+        }
+    }
+  }
+
+  /**
     * Returns the built-in normalization rules that are defined by the environment.
     */
   protected def getBuiltInNormRuleSet: RuleSet = FlinkRuleSets.DATASTREAM_NORM_RULES
@@ -221,6 +243,11 @@ abstract class StreamTableEnvironment(
   protected def getBuiltInOptRuleSet: RuleSet = FlinkRuleSets.DATASTREAM_OPT_RULES
 
   /**
+    * Returns the built-in decoration rules that are defined by the environment.
+    */
+  protected def getBuiltInDecoRuleSet: RuleSet = FlinkRuleSets.DATASTREAM_DECO_RULES
+
+  /**
     * Generates the optimized [[RelNode]] tree from the original relational node tree.
     *
     * @param relNode The root node of the relational expression tree.
@@ -248,7 +275,14 @@ abstract class StreamTableEnvironment(
       normalizedPlan
     }
 
-    optimizedPlan
+    // 4. decorate the optimized plan
+    val decoRuleSet = getDecoRuleSet
+    val decoratedPlan = if (decoRuleSet.iterator().hasNext) {
+      runHepPlanner(HepMatchOrder.BOTTOM_UP, decoRuleSet, optimizedPlan, optimizedPlan.getTraitSet)
+    } else {
+      optimizedPlan
+    }
+    decoratedPlan
   }
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6949c8c7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
index 65a61b2..ba8df81 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
@@ -31,15 +31,36 @@ import scala.collection.JavaConverters._
   * Builder for creating a Calcite configuration.
   */
 class CalciteConfigBuilder {
+
+  /**
+    * Defines the normalization rule set. Normalization rules are dedicated for rewriting
+    * predicated logical plan before volcano optimization.
+    */
   private var replaceNormRules: Boolean = false
   private var normRuleSets: List[RuleSet] = Nil
 
+  /**
+    * Defines the optimization rule set. Optimization rules are used during volcano optimization.
+    */
   private var replaceOptRules: Boolean = false
   private var optRuleSets: List[RuleSet] = Nil
 
+  /**
+    * Defines the decoration rule set. Decoration rules are dedicated for rewriting predicated
+    * logical plan after volcano optimization.
+    */
+  private var replaceDecoRules: Boolean = false
+  private var decoRuleSets: List[RuleSet] = Nil
+
+  /**
+    * Defines the SQL operator tables.
+    */
   private var replaceOperatorTable: Boolean = false
   private var operatorTables: List[SqlOperatorTable] = Nil
 
+  /**
+    * Defines a SQL parser configuration.
+    */
   private var replaceSqlParserConfig: Option[SqlParser.Config] = None
 
   /**
@@ -81,6 +102,32 @@ class CalciteConfigBuilder {
   }
 
   /**
+    * Replaces the built-in decoration rule set with the given rule set.
+    *
+    * The decoration rules are applied after the cost-based optimization phase.
+    * The decoration phase allows to rewrite the optimized plan and is not cost-based.
+    *
+    */
+  def replaceDecoRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(replaceRuleSet)
+    decoRuleSets = List(replaceRuleSet)
+    replaceDecoRules = true
+    this
+  }
+
+  /**
+    * Appends the given decoration rule set to the built-in rule set.
+    *
+    * The decoration rules are applied after the cost-based optimization phase.
+    * The decoration phase allows to rewrite the optimized plan and is not cost-based.
+    */
+  def addDecoRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(addedRuleSet)
+    decoRuleSets = addedRuleSet :: decoRuleSets
+    this
+  }
+
+  /**
     * Replaces the built-in SQL operator table with the given table.
     */
   def replaceSqlOperatorTable(replaceSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
@@ -113,35 +160,39 @@ class CalciteConfigBuilder {
     val replacesNormRuleSet: Boolean,
     val getOptRuleSet: Option[RuleSet],
     val replacesOptRuleSet: Boolean,
+    val getDecoRuleSet: Option[RuleSet],
+    val replacesDecoRuleSet: Boolean,
     val getSqlOperatorTable: Option[SqlOperatorTable],
     val replacesSqlOperatorTable: Boolean,
     val getSqlParserConfig: Option[SqlParser.Config])
     extends CalciteConfig
 
+
   /**
-    * Builds a new [[CalciteConfig]].
+    * Convert the [[RuleSet]] List to [[Option]] type
     */
-  def build(): CalciteConfig = new CalciteConfigImpl(
-    normRuleSets match {
+  private def getRuleSet(inputRuleSet: List[RuleSet]): Option[RuleSet] = {
+    inputRuleSet match {
       case Nil => None
       case h :: Nil => Some(h)
       case _ =>
         // concat rule sets
         val concatRules =
-          normRuleSets.foldLeft(Nil: Iterable[RelOptRule])((c, r) => r.asScala ++ c)
+          inputRuleSet.foldLeft(Nil: Iterable[RelOptRule])((c, r) => r.asScala ++ c)
         Some(RuleSets.ofList(concatRules.asJava))
-    },
+    }
+  }
+
+  /**
+    * Builds a new [[CalciteConfig]].
+    */
+  def build(): CalciteConfig = new CalciteConfigImpl(
+    getRuleSet(normRuleSets),
     replaceNormRules,
-    optRuleSets match {
-      case Nil => None
-      case h :: Nil => Some(h)
-      case _ =>
-        // concat rule sets
-        val concatRules =
-          optRuleSets.foldLeft(Nil: Iterable[RelOptRule])((c, r) => r.asScala ++ c)
-        Some(RuleSets.ofList(concatRules.asJava))
-    },
+    getRuleSet(optRuleSets),
     replaceOptRules,
+    getRuleSet(decoRuleSets),
+    replaceDecoRules,
     operatorTables match {
       case Nil => None
       case h :: Nil => Some(h)
@@ -179,6 +230,16 @@ trait CalciteConfig {
   def getOptRuleSet: Option[RuleSet]
 
   /**
+    * Returns whether this configuration replaces the built-in decoration rule set.
+    */
+  def replacesDecoRuleSet: Boolean
+
+  /**
+    * Returns a custom decoration rule set.
+    */
+  def getDecoRuleSet: Option[RuleSet]
+
+  /**
     * Returns whether this configuration replaces the built-in SQL operator table.
     */
   def replacesSqlOperatorTable: Boolean

http://git-wip-us.apache.org/repos/asf/flink/blob/6949c8c7/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
index 952ee34..1301c8d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -23,7 +23,6 @@ import org.apache.calcite.tools.{RuleSet, RuleSets}
 import org.apache.flink.table.calcite.rules.{FlinkAggregateExpandDistinctAggregatesRule, FlinkAggregateJoinTransposeRule}
 import org.apache.flink.table.plan.rules.dataSet._
 import org.apache.flink.table.plan.rules.datastream._
-import org.apache.flink.table.plan.rules.datastream.{DataStreamCalcRule, DataStreamScanRule, DataStreamUnionRule}
 
 object FlinkRuleSets {
 
@@ -186,4 +185,12 @@ object FlinkRuleSets {
       PushFilterIntoStreamTableSourceScanRule.INSTANCE
   )
 
+  /**
+    * RuleSet to decorate plans for stream / DataStream execution
+    */
+  val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList(
+    // rules
+
+  )
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6949c8c7/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
index 6c07e28..d0de8fa 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
@@ -39,6 +39,9 @@ class CalciteConfigBuilderTest {
 
     assertFalse(cc.replacesOptRuleSet)
     assertFalse(cc.getOptRuleSet.isDefined)
+
+    assertFalse(cc.replacesDecoRuleSet)
+    assertFalse(cc.getDecoRuleSet.isDefined)
   }
 
   @Test
@@ -47,6 +50,7 @@ class CalciteConfigBuilderTest {
     val cc: CalciteConfig = new CalciteConfigBuilder()
       .addNormRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
       .replaceOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
       .build()
 
     assertFalse(cc.replacesNormRuleSet)
@@ -54,6 +58,9 @@ class CalciteConfigBuilderTest {
 
     assertTrue(cc.replacesOptRuleSet)
     assertTrue(cc.getOptRuleSet.isDefined)
+
+    assertTrue(cc.replacesDecoRuleSet)
+    assertTrue(cc.getDecoRuleSet.isDefined)
   }
 
   @Test
@@ -181,6 +188,68 @@ class CalciteConfigBuilderTest {
   }
 
   @Test
+  def testReplaceDecorationRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .build()
+
+    assertEquals(true, cc.replacesDecoRuleSet)
+    assertTrue(cc.getDecoRuleSet.isDefined)
+    val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
+    assertEquals(1, cSet.size)
+    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
+  }
+
+  @Test
+  def testReplaceDecorationAddRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.PROJECT_INSTANCE))
+      .build()
+
+    assertEquals(true, cc.replacesDecoRuleSet)
+    assertTrue(cc.getDecoRuleSet.isDefined)
+    val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
+    assertEquals(2, cSet.size)
+    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
+    assertTrue(cSet.contains(ReduceExpressionsRule.PROJECT_INSTANCE))
+  }
+
+  @Test
+  def testAddDecorationRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .build()
+
+    assertEquals(false, cc.replacesDecoRuleSet)
+    assertTrue(cc.getDecoRuleSet.isDefined)
+    val cSet = cc.getDecoRuleSet.get.iterator().asScala.toSet
+    assertEquals(1, cSet.size)
+    assertTrue(cSet.contains(ReduceExpressionsRule.FILTER_INSTANCE))
+  }
+
+  @Test
+  def testAddAddDecorationRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE))
+      .addDecoRuleSet(RuleSets.ofList(ReduceExpressionsRule.PROJECT_INSTANCE,
+                                      ReduceExpressionsRule.CALC_INSTANCE))
+      .build()
+
+    assertEquals(false, cc.replacesDecoRuleSet)
+    assertTrue(cc.getDecoRuleSet.isDefined)
+    val cList = cc.getDecoRuleSet.get.iterator().asScala.toList
+    assertEquals(3, cList.size)
+    assertEquals(cList.head, ReduceExpressionsRule.FILTER_INSTANCE)
+    assertEquals(cList(1), ReduceExpressionsRule.PROJECT_INSTANCE)
+    assertEquals(cList(2), ReduceExpressionsRule.CALC_INSTANCE)
+  }
+
+  @Test
   def testDefaultOperatorTable(): Unit = {
 
     val cc: CalciteConfig = new CalciteConfigBuilder()


[37/50] [abbrv] flink git commit: [FLINK-6034] [checkpoints] Introduce KeyedStateHandle abstraction for the snapshots in keyed streams

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResult.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResult.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResult.java
index b1c94cb..8aa76a5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResult.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResult.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.api.operators;
 
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateUtil;
 import org.apache.flink.util.ExceptionUtils;
@@ -30,8 +30,8 @@ import java.util.concurrent.RunnableFuture;
  */
 public class OperatorSnapshotResult {
 
-	private RunnableFuture<KeyGroupsStateHandle> keyedStateManagedFuture;
-	private RunnableFuture<KeyGroupsStateHandle> keyedStateRawFuture;
+	private RunnableFuture<KeyedStateHandle> keyedStateManagedFuture;
+	private RunnableFuture<KeyedStateHandle> keyedStateRawFuture;
 	private RunnableFuture<OperatorStateHandle> operatorStateManagedFuture;
 	private RunnableFuture<OperatorStateHandle> operatorStateRawFuture;
 
@@ -40,8 +40,8 @@ public class OperatorSnapshotResult {
 	}
 
 	public OperatorSnapshotResult(
-			RunnableFuture<KeyGroupsStateHandle> keyedStateManagedFuture,
-			RunnableFuture<KeyGroupsStateHandle> keyedStateRawFuture,
+			RunnableFuture<KeyedStateHandle> keyedStateManagedFuture,
+			RunnableFuture<KeyedStateHandle> keyedStateRawFuture,
 			RunnableFuture<OperatorStateHandle> operatorStateManagedFuture,
 			RunnableFuture<OperatorStateHandle> operatorStateRawFuture) {
 		this.keyedStateManagedFuture = keyedStateManagedFuture;
@@ -50,19 +50,19 @@ public class OperatorSnapshotResult {
 		this.operatorStateRawFuture = operatorStateRawFuture;
 	}
 
-	public RunnableFuture<KeyGroupsStateHandle> getKeyedStateManagedFuture() {
+	public RunnableFuture<KeyedStateHandle> getKeyedStateManagedFuture() {
 		return keyedStateManagedFuture;
 	}
 
-	public void setKeyedStateManagedFuture(RunnableFuture<KeyGroupsStateHandle> keyedStateManagedFuture) {
+	public void setKeyedStateManagedFuture(RunnableFuture<KeyedStateHandle> keyedStateManagedFuture) {
 		this.keyedStateManagedFuture = keyedStateManagedFuture;
 	}
 
-	public RunnableFuture<KeyGroupsStateHandle> getKeyedStateRawFuture() {
+	public RunnableFuture<KeyedStateHandle> getKeyedStateRawFuture() {
 		return keyedStateRawFuture;
 	}
 
-	public void setKeyedStateRawFuture(RunnableFuture<KeyGroupsStateHandle> keyedStateRawFuture) {
+	public void setKeyedStateRawFuture(RunnableFuture<KeyedStateHandle> keyedStateRawFuture) {
 		this.keyedStateRawFuture = keyedStateRawFuture;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
index 7abf8d9..30d07b7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
@@ -21,7 +21,7 @@ package org.apache.flink.streaming.runtime.tasks;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.state.ChainedStateHandle;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateHandles;
@@ -42,16 +42,16 @@ public class OperatorStateHandles {
 
 	private final StreamStateHandle legacyOperatorState;
 
-	private final Collection<KeyGroupsStateHandle> managedKeyedState;
-	private final Collection<KeyGroupsStateHandle> rawKeyedState;
+	private final Collection<KeyedStateHandle> managedKeyedState;
+	private final Collection<KeyedStateHandle> rawKeyedState;
 	private final Collection<OperatorStateHandle> managedOperatorState;
 	private final Collection<OperatorStateHandle> rawOperatorState;
 
 	public OperatorStateHandles(
 			int operatorChainIndex,
 			StreamStateHandle legacyOperatorState,
-			Collection<KeyGroupsStateHandle> managedKeyedState,
-			Collection<KeyGroupsStateHandle> rawKeyedState,
+			Collection<KeyedStateHandle> managedKeyedState,
+			Collection<KeyedStateHandle> rawKeyedState,
 			Collection<OperatorStateHandle> managedOperatorState,
 			Collection<OperatorStateHandle> rawOperatorState) {
 
@@ -83,11 +83,11 @@ public class OperatorStateHandles {
 		return legacyOperatorState;
 	}
 
-	public Collection<KeyGroupsStateHandle> getManagedKeyedState() {
+	public Collection<KeyedStateHandle> getManagedKeyedState() {
 		return managedKeyedState;
 	}
 
-	public Collection<KeyGroupsStateHandle> getRawKeyedState() {
+	public Collection<KeyedStateHandle> getRawKeyedState() {
 		return rawKeyedState;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 76b2b98..11e8e0d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -37,7 +37,7 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
@@ -849,8 +849,8 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 		private final List<OperatorSnapshotResult> snapshotInProgressList;
 
-		private RunnableFuture<KeyGroupsStateHandle> futureKeyedBackendStateHandles;
-		private RunnableFuture<KeyGroupsStateHandle> futureKeyedStreamStateHandles;
+		private RunnableFuture<KeyedStateHandle> futureKeyedBackendStateHandles;
+		private RunnableFuture<KeyedStateHandle> futureKeyedStreamStateHandles;
 
 		private List<StreamStateHandle> nonPartitionedStateHandles;
 
@@ -892,8 +892,8 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 		public void run() {
 			try {
 				// Keyed state handle future, currently only one (the head) operator can have this
-				KeyGroupsStateHandle keyedStateHandleBackend = FutureUtil.runIfNotDoneAndGet(futureKeyedBackendStateHandles);
-				KeyGroupsStateHandle keyedStateHandleStream = FutureUtil.runIfNotDoneAndGet(futureKeyedStreamStateHandles);
+				KeyedStateHandle keyedStateHandleBackend = FutureUtil.runIfNotDoneAndGet(futureKeyedBackendStateHandles);
+				KeyedStateHandle keyedStateHandleStream = FutureUtil.runIfNotDoneAndGet(futureKeyedStreamStateHandles);
 
 				List<OperatorStateHandle> operatorStatesBackend = new ArrayList<>(snapshotInProgressList.size());
 				List<OperatorStateHandle> operatorStatesStream = new ArrayList<>(snapshotInProgressList.size());
@@ -987,8 +987,8 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 				ChainedStateHandle<StreamStateHandle> chainedNonPartitionedOperatorsState,
 				ChainedStateHandle<OperatorStateHandle> chainedOperatorStateBackend,
 				ChainedStateHandle<OperatorStateHandle> chainedOperatorStateStream,
-				KeyGroupsStateHandle keyedStateHandleBackend,
-				KeyGroupsStateHandle keyedStateHandleStream) {
+				KeyedStateHandle keyedStateHandleBackend,
+				KeyedStateHandle keyedStateHandleStream) {
 
 			boolean hasAnyState = keyedStateHandleBackend != null
 					|| keyedStateHandleStream != null

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
index eeee8dc..8f42c1a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
@@ -51,7 +51,7 @@ import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
@@ -559,11 +559,11 @@ public class AbstractStreamOperatorTest {
 
 		final CloseableRegistry closeableRegistry = new CloseableRegistry();
 
-		RunnableFuture<KeyGroupsStateHandle> futureKeyGroupStateHandle = mock(RunnableFuture.class);
+		RunnableFuture<KeyedStateHandle> futureKeyedStateHandle = mock(RunnableFuture.class);
 		RunnableFuture<OperatorStateHandle> futureOperatorStateHandle = mock(RunnableFuture.class);
 
 		StateSnapshotContextSynchronousImpl context = mock(StateSnapshotContextSynchronousImpl.class);
-		when(context.getKeyedStateStreamFuture()).thenReturn(futureKeyGroupStateHandle);
+		when(context.getKeyedStateStreamFuture()).thenReturn(futureKeyedStateHandle);
 		when(context.getOperatorStateStreamFuture()).thenReturn(futureOperatorStateHandle);
 
 		OperatorSnapshotResult operatorSnapshotResult = spy(new OperatorSnapshotResult());
@@ -609,9 +609,9 @@ public class AbstractStreamOperatorTest {
 		verify(context).close();
 		verify(operatorSnapshotResult).cancel();
 
-		verify(futureKeyGroupStateHandle).cancel(anyBoolean());
+		verify(futureKeyedStateHandle).cancel(anyBoolean());
 		verify(futureOperatorStateHandle).cancel(anyBoolean());
-		verify(futureKeyGroupStateHandle).cancel(anyBoolean());
+		verify(futureKeyedStateHandle).cancel(anyBoolean());
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResultTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResultTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResultTest.java
index 490df52..f57eed1 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResultTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotResultTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.api.operators;
 
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -41,12 +41,12 @@ public class OperatorSnapshotResultTest extends TestLogger {
 
 		operatorSnapshotResult.cancel();
 
-		KeyGroupsStateHandle keyedManagedStateHandle = mock(KeyGroupsStateHandle.class);
-		RunnableFuture<KeyGroupsStateHandle> keyedStateManagedFuture = mock(RunnableFuture.class);
+		KeyedStateHandle keyedManagedStateHandle = mock(KeyedStateHandle.class);
+		RunnableFuture<KeyedStateHandle> keyedStateManagedFuture = mock(RunnableFuture.class);
 		when(keyedStateManagedFuture.get()).thenReturn(keyedManagedStateHandle);
 
-		KeyGroupsStateHandle keyedRawStateHandle = mock(KeyGroupsStateHandle.class);
-		RunnableFuture<KeyGroupsStateHandle> keyedStateRawFuture = mock(RunnableFuture.class);
+		KeyedStateHandle keyedRawStateHandle = mock(KeyedStateHandle.class);
+		RunnableFuture<KeyedStateHandle> keyedStateRawFuture = mock(RunnableFuture.class);
 		when(keyedStateRawFuture.get()).thenReturn(keyedRawStateHandle);
 
 		OperatorStateHandle operatorManagedStateHandle = mock(OperatorStateHandle.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java
index 963c42c..8e0edfc 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateInitializationContextImpl;
 import org.apache.flink.runtime.state.StatePartitionStreamProvider;
@@ -75,7 +76,7 @@ public class StateInitializationContextImplTest {
 
 		ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos(64);
 
-		List<KeyGroupsStateHandle> keyGroupsStateHandles = new ArrayList<>(NUM_HANDLES);
+		List<KeyedStateHandle> keyedStateHandles = new ArrayList<>(NUM_HANDLES);
 		int prev = 0;
 		for (int i = 0; i < NUM_HANDLES; ++i) {
 			out.reset();
@@ -91,10 +92,10 @@ public class StateInitializationContextImplTest {
 				++writtenKeyGroups;
 			}
 
-			KeyGroupsStateHandle handle =
+			KeyedStateHandle handle =
 					new KeyGroupsStateHandle(offsets, new ByteStateHandleCloseChecking("kg-" + i, out.toByteArray()));
 
-			keyGroupsStateHandles.add(handle);
+			keyedStateHandles.add(handle);
 		}
 
 		List<OperatorStateHandle> operatorStateHandles = new ArrayList<>(NUM_HANDLES);
@@ -125,7 +126,7 @@ public class StateInitializationContextImplTest {
 						true,
 						stateStore,
 						mock(KeyedStateStore.class),
-						keyGroupsStateHandles,
+						keyedStateHandles,
 						operatorStateHandles,
 						closableRegistry);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index 58cfefd..4435247 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -51,6 +51,7 @@ import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StreamStateHandle;
@@ -186,8 +187,8 @@ public class InterruptSensitiveRestoreTest {
 
 
 		ChainedStateHandle<StreamStateHandle> operatorState = null;
-		List<KeyGroupsStateHandle> keyGroupStateFromBackend = Collections.emptyList();
-		List<KeyGroupsStateHandle> keyGroupStateFromStream = Collections.emptyList();
+		List<KeyedStateHandle> keyedStateFromBackend = Collections.emptyList();
+		List<KeyedStateHandle> keyedStateFromStream = Collections.emptyList();
 		List<Collection<OperatorStateHandle>> operatorStateBackend = Collections.emptyList();
 		List<Collection<OperatorStateHandle>> operatorStateStream = Collections.emptyList();
 
@@ -201,8 +202,8 @@ public class InterruptSensitiveRestoreTest {
 		Collection<OperatorStateHandle> operatorStateHandles =
 				Collections.singletonList(new OperatorStateHandle(operatorStateMetadata, state));
 
-		List<KeyGroupsStateHandle> keyGroupsStateHandles =
-				Collections.singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state));
+		List<KeyedStateHandle> keyedStateHandles =
+				Collections.<KeyedStateHandle>singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state));
 
 		switch (mode) {
 			case OPERATOR_MANAGED:
@@ -212,10 +213,10 @@ public class InterruptSensitiveRestoreTest {
 				operatorStateStream = Collections.singletonList(operatorStateHandles);
 				break;
 			case KEYED_MANAGED:
-				keyGroupStateFromBackend = keyGroupsStateHandles;
+				keyedStateFromBackend = keyedStateHandles;
 				break;
 			case KEYED_RAW:
-				keyGroupStateFromStream = keyGroupsStateHandles;
+				keyedStateFromStream = keyedStateHandles;
 				break;
 			case LEGACY:
 				operatorState = new ChainedStateHandle<>(Collections.singletonList(state));
@@ -228,8 +229,8 @@ public class InterruptSensitiveRestoreTest {
 			operatorState,
 			operatorStateBackend,
 			operatorStateStream,
-			keyGroupStateFromBackend,
-			keyGroupStateFromStream);
+			keyedStateFromBackend,
+			keyedStateFromStream);
 
 		JobInformation jobInformation = new JobInformation(
 			new JobID(),

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index d7e3d6c..f34522b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -61,7 +61,7 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.DoneFuture;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackendFactory;
@@ -458,8 +458,8 @@ public class StreamTaskTest extends TestLogger {
 
 		StreamOperator<?> streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
 
-		KeyGroupsStateHandle managedKeyedStateHandle = mock(KeyGroupsStateHandle.class);
-		KeyGroupsStateHandle rawKeyedStateHandle = mock(KeyGroupsStateHandle.class);
+		KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class);
+		KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class);
 		OperatorStateHandle managedOperatorStateHandle = mock(OperatorStateHandle.class);
 		OperatorStateHandle rawOperatorStateHandle = mock(OperatorStateHandle.class);
 
@@ -563,8 +563,8 @@ public class StreamTaskTest extends TestLogger {
 					(ChainedStateHandle<StreamStateHandle>)invocation.getArguments()[0],
 					(ChainedStateHandle<OperatorStateHandle>)invocation.getArguments()[1],
 					(ChainedStateHandle<OperatorStateHandle>)invocation.getArguments()[2],
-					(KeyGroupsStateHandle)invocation.getArguments()[3],
-					(KeyGroupsStateHandle)invocation.getArguments()[4]);
+					(KeyedStateHandle)invocation.getArguments()[3],
+					(KeyedStateHandle)invocation.getArguments()[4]);
 			}
 		});
 
@@ -574,8 +574,8 @@ public class StreamTaskTest extends TestLogger {
 
 		StreamOperator<?> streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
 
-		KeyGroupsStateHandle managedKeyedStateHandle = mock(KeyGroupsStateHandle.class);
-		KeyGroupsStateHandle rawKeyedStateHandle = mock(KeyGroupsStateHandle.class);
+		KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class);
+		KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class);
 		OperatorStateHandle managedOperatorStateHandle = mock(OperatorStateHandle.class);
 		OperatorStateHandle rawOperatorStateHandle = mock(OperatorStateHandle.class);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
index 945103c..912d579 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
@@ -41,6 +41,7 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
@@ -318,7 +319,7 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 
 		StreamStateHandle stateHandle = SavepointV0Serializer.convertOperatorAndFunctionState(state);
 
-		List<KeyGroupsStateHandle> keyGroupStatesList = new ArrayList<>();
+		List<KeyedStateHandle> keyGroupStatesList = new ArrayList<>();
 		if (state.getKvStates() != null) {
 			KeyGroupsStateHandle keyedStateHandle = SavepointV0Serializer.convertKeyedBackendState(
 					state.getKvStates(),
@@ -331,7 +332,7 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 		initializeState(new OperatorStateHandles(0,
 				stateHandle,
 				keyGroupStatesList,
-				Collections.<KeyGroupsStateHandle>emptyList(),
+				Collections.<KeyedStateHandle>emptyList(),
 				Collections.<OperatorStateHandle>emptyList(),
 				Collections.<OperatorStateHandle>emptyList()));
 	}
@@ -364,16 +365,16 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 			KeyGroupRange localKeyGroupRange =
 					keyGroupPartitions.get(subtaskIndex);
 
-			List<KeyGroupsStateHandle> localManagedKeyGroupState = null;
+			List<KeyedStateHandle> localManagedKeyGroupState = null;
 			if (operatorStateHandles.getManagedKeyedState() != null) {
-				localManagedKeyGroupState = StateAssignmentOperation.getKeyGroupsStateHandles(
+				localManagedKeyGroupState = StateAssignmentOperation.getKeyedStateHandles(
 						operatorStateHandles.getManagedKeyedState(),
 						localKeyGroupRange);
 			}
 
-			List<KeyGroupsStateHandle> localRawKeyGroupState = null;
+			List<KeyedStateHandle> localRawKeyGroupState = null;
 			if (operatorStateHandles.getRawKeyedState() != null) {
-				localRawKeyGroupState = StateAssignmentOperation.getKeyGroupsStateHandles(
+				localRawKeyGroupState = StateAssignmentOperation.getKeyedStateHandles(
 						operatorStateHandles.getRawKeyedState(),
 						localKeyGroupRange);
 			}
@@ -442,15 +443,15 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 		List<OperatorStateHandle> mergedManagedOperatorState = new ArrayList<>(handles.length);
 		List<OperatorStateHandle> mergedRawOperatorState = new ArrayList<>(handles.length);
 
-		List<KeyGroupsStateHandle> mergedManagedKeyedState = new ArrayList<>(handles.length);
-		List<KeyGroupsStateHandle> mergedRawKeyedState = new ArrayList<>(handles.length);
+		List<KeyedStateHandle> mergedManagedKeyedState = new ArrayList<>(handles.length);
+		List<KeyedStateHandle> mergedRawKeyedState = new ArrayList<>(handles.length);
 
 		for (OperatorStateHandles handle: handles) {
 
 			Collection<OperatorStateHandle> managedOperatorState = handle.getManagedOperatorState();
 			Collection<OperatorStateHandle> rawOperatorState = handle.getRawOperatorState();
-			Collection<KeyGroupsStateHandle> managedKeyedState = handle.getManagedKeyedState();
-			Collection<KeyGroupsStateHandle> rawKeyedState = handle.getRawKeyedState();
+			Collection<KeyedStateHandle> managedKeyedState = handle.getManagedKeyedState();
+			Collection<KeyedStateHandle> rawKeyedState = handle.getRawKeyedState();
 
 			if (managedOperatorState != null) {
 				mergedManagedOperatorState.addAll(managedOperatorState);
@@ -502,8 +503,8 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 			timestamp,
 			CheckpointOptions.forFullCheckpoint());
 
-		KeyGroupsStateHandle keyedManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateManagedFuture());
-		KeyGroupsStateHandle keyedRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateRawFuture());
+		KeyedStateHandle keyedManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateManagedFuture());
+		KeyedStateHandle keyedRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateRawFuture());
 
 		OperatorStateHandle opManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateManagedFuture());
 		OperatorStateHandle opRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateRawFuture());

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
index d45ae21..d9c7387 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
@@ -65,7 +66,7 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 
 	// when we restore we keep the state here so that we can call restore
 	// when the operator requests the keyed state backend
-	private List<KeyGroupsStateHandle> restoredKeyedState = null;
+	private List<KeyedStateHandle> restoredKeyedState = null;
 
 	public KeyedOneInputStreamOperatorTestHarness(
 			OneInputStreamOperator<IN, OUT> operator,
@@ -144,7 +145,7 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 		}
 
 		if (keyedStateBackend != null) {
-			RunnableFuture<KeyGroupsStateHandle> keyedSnapshotRunnable = keyedStateBackend.snapshot(
+			RunnableFuture<KeyedStateHandle> keyedSnapshotRunnable = keyedStateBackend.snapshot(
 					checkpointId,
 					timestamp,
 					streamFactory,
@@ -177,14 +178,14 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 			byte keyedStatePresent = (byte) inStream.read();
 			if (keyedStatePresent == 1) {
 				ObjectInputStream ois = new ObjectInputStream(inStream);
-				this.restoredKeyedState = Collections.singletonList((KeyGroupsStateHandle) ois.readObject());
+				this.restoredKeyedState = Collections.singletonList((KeyedStateHandle) ois.readObject());
 			}
 		}
 	}
 
 
-	private static boolean hasMigrationHandles(Collection<KeyGroupsStateHandle> allKeyGroupsHandles) {
-		for (KeyGroupsStateHandle handle : allKeyGroupsHandles) {
+	private static boolean hasMigrationHandles(Collection<KeyedStateHandle> allKeyGroupsHandles) {
+		for (KeyedStateHandle handle : allKeyGroupsHandles) {
 			if (handle instanceof Migration) {
 				return true;
 			}
@@ -225,17 +226,17 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 					keyGroupPartitions.get(subtaskIndex);
 
 			restoredKeyedState = null;
-			Collection<KeyGroupsStateHandle> managedKeyedState = operatorStateHandles.getManagedKeyedState();
+			Collection<KeyedStateHandle> managedKeyedState = operatorStateHandles.getManagedKeyedState();
 			if (managedKeyedState != null) {
 
 				// if we have migration handles, don't reshuffle state and preserve
 				// the migration tag
 				if (hasMigrationHandles(managedKeyedState)) {
-					List<KeyGroupsStateHandle> result = new ArrayList<>(managedKeyedState.size());
+					List<KeyedStateHandle> result = new ArrayList<>(managedKeyedState.size());
 					result.addAll(managedKeyedState);
 					restoredKeyedState = result;
 				} else {
-					restoredKeyedState = StateAssignmentOperation.getKeyGroupsStateHandles(
+					restoredKeyedState = StateAssignmentOperation.getKeyedStateHandles(
 							managedKeyedState,
 							localKeyGroupRange);
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/cd552741/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedTwoInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedTwoInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedTwoInputStreamOperatorTestHarness.java
index 8e76f70..41a083a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedTwoInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedTwoInputStreamOperatorTestHarness.java
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
 import org.mockito.invocation.InvocationOnMock;
@@ -50,7 +51,7 @@ public class KeyedTwoInputStreamOperatorTestHarness<K, IN1, IN2, OUT>
 
 	// when we restore we keep the state here so that we can call restore
 	// when the operator requests the keyed state backend
-	private Collection<KeyGroupsStateHandle> restoredKeyedState = null;
+	private Collection<KeyedStateHandle> restoredKeyedState = null;
 
 	public KeyedTwoInputStreamOperatorTestHarness(
 			TwoInputStreamOperator<IN1, IN2, OUT> operator,


[26/50] [abbrv] flink git commit: [FLINK-5698] [table] Add NestedFieldsProjectableTableSource interface.

Posted by fh...@apache.org.
[FLINK-5698] [table] Add NestedFieldsProjectableTableSource interface.

This closes #3269.


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

Branch: refs/heads/table-retraction
Commit: 5c37e55c83f854c1a9eb7bd7438b378b8c4b0a9f
Parents: cac9fa0
Author: tonycox <an...@epam.com>
Authored: Mon Feb 6 16:32:45 2017 +0400
Committer: Fabian Hueske <fh...@apache.org>
Committed: Sat Mar 25 00:32:31 2017 +0100

----------------------------------------------------------------------
 ...PushProjectIntoTableSourceScanRuleBase.scala |  17 +-
 .../table/plan/util/RexProgramExtractor.scala   |  81 +++++++++
 .../NestedFieldsProjectableTableSource.scala    |  54 ++++++
 .../plan/util/RexProgramExtractorTest.scala     | 181 ++++++++++++++++++-
 .../flink/table/utils/InputTypeBuilder.scala    |  53 ++++++
 5 files changed, 380 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5c37e55c/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushProjectIntoTableSourceScanRuleBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushProjectIntoTableSourceScanRuleBase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushProjectIntoTableSourceScanRuleBase.scala
index 9f9c805..1e75971 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushProjectIntoTableSourceScanRuleBase.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/PushProjectIntoTableSourceScanRuleBase.scala
@@ -23,7 +23,7 @@ import org.apache.calcite.rel.core.Calc
 import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.plan.nodes.TableSourceScan
 import org.apache.flink.table.plan.util.{RexProgramExtractor, RexProgramRewriter}
-import org.apache.flink.table.sources.ProjectableTableSource
+import org.apache.flink.table.sources.{NestedFieldsProjectableTableSource, ProjectableTableSource}
 
 trait PushProjectIntoTableSourceScanRuleBase {
 
@@ -35,9 +35,18 @@ trait PushProjectIntoTableSourceScanRuleBase {
     val usedFields = RexProgramExtractor.extractRefInputFields(calc.getProgram)
 
     // if no fields can be projected, we keep the original plan.
-    if (TableEnvironment.getFieldNames(scan.tableSource).length != usedFields.length) {
-      val originTableSource = scan.tableSource.asInstanceOf[ProjectableTableSource[_]]
-      val newTableSource = originTableSource.projectFields(usedFields)
+    val source = scan.tableSource
+    if (TableEnvironment.getFieldNames(source).length != usedFields.length) {
+
+      val newTableSource = source match {
+        case nested: NestedFieldsProjectableTableSource[_] =>
+          val nestedFields = RexProgramExtractor
+            .extractRefNestedInputFields(calc.getProgram, usedFields)
+          nested.projectNestedFields(usedFields, nestedFields)
+        case projecting: ProjectableTableSource[_] =>
+          projecting.projectFields(usedFields)
+      }
+
       val newScan = scan.copy(scan.getTraitSet, newTableSource)
       val newCalcProgram = RexProgramRewriter.rewriteWithFieldProjection(
         calc.getProgram,

http://git-wip-us.apache.org/repos/asf/flink/blob/5c37e55c/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala
index 433a35b..a042f55 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala
@@ -92,6 +92,26 @@ object RexProgramExtractor {
       case _ => (Array.empty, Array.empty)
     }
   }
+
+  /**
+    * Extracts the name of nested input fields accessed by the RexProgram and returns the
+    * prefix of the accesses.
+    *
+    * @param rexProgram The RexProgram to analyze
+    * @return The full names of accessed input fields. e.g. field.subfield
+    */
+  def extractRefNestedInputFields(
+      rexProgram: RexProgram, usedFields: Array[Int]): Array[Array[String]] = {
+
+    val visitor = new RefFieldAccessorVisitor(usedFields)
+    rexProgram.getProjectList.foreach(exp => rexProgram.expandLocalRef(exp).accept(visitor))
+
+    val condition = rexProgram.getCondition
+    if (condition != null) {
+      rexProgram.expandLocalRef(condition).accept(visitor)
+    }
+    visitor.getProjectedFields
+  }
 }
 
 /**
@@ -181,3 +201,64 @@ class RexNodeToExpressionConverter(
   }
 
 }
+
+/**
+  * A RexVisitor to extract used nested input fields
+  */
+class RefFieldAccessorVisitor(usedFields: Array[Int]) extends RexVisitorImpl[Unit](true) {
+
+  private val projectedFields: Array[Array[String]] = Array.fill(usedFields.length)(Array.empty)
+
+  private val order: Map[Int, Int] = usedFields.zipWithIndex.toMap
+
+  /** Returns the prefix of the nested field accesses */
+  def getProjectedFields: Array[Array[String]] = {
+
+    projectedFields.map { nestedFields =>
+      // sort nested field accesses
+      val sorted = nestedFields.sorted
+      // get prefix field accesses
+      val prefixAccesses = sorted.foldLeft(Nil: List[String]) {
+        (prefixAccesses, nestedAccess) => prefixAccesses match {
+              // first access => add access
+            case Nil => List[String](nestedAccess)
+              // top-level access already found => return top-level access
+            case head :: Nil if head.equals("*") => prefixAccesses
+              // access is top-level access => return top-level access
+            case _ :: _ if nestedAccess.equals("*") => List("*")
+            // previous access is not prefix of this access => add access
+            case head :: _ if !nestedAccess.startsWith(head) =>
+              nestedAccess :: prefixAccesses
+              // previous access is a prefix of this access => do not add access
+            case _ => prefixAccesses
+          }
+      }
+      prefixAccesses.toArray
+    }
+  }
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): Unit = {
+    def internalVisit(fieldAccess: RexFieldAccess): (Int, String) = {
+      fieldAccess.getReferenceExpr match {
+        case ref: RexInputRef =>
+          (ref.getIndex, fieldAccess.getField.getName)
+        case fac: RexFieldAccess =>
+          val (i, n) = internalVisit(fac)
+          (i, s"$n.${fieldAccess.getField.getName}")
+      }
+    }
+    val (index, fullName) = internalVisit(fieldAccess)
+    val outputIndex = order.getOrElse(index, -1)
+    val fields: Array[String] = projectedFields(outputIndex)
+    projectedFields(outputIndex) = fields :+ fullName
+  }
+
+  override def visitInputRef(inputRef: RexInputRef): Unit = {
+    val outputIndex = order.getOrElse(inputRef.getIndex, -1)
+    val fields: Array[String] = projectedFields(outputIndex)
+    projectedFields(outputIndex) = fields :+ "*"
+  }
+
+  override def visitCall(call: RexCall): Unit =
+    call.operands.foreach(operand => operand.accept(this))
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5c37e55c/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.scala
new file mode 100644
index 0000000..a10187b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.scala
@@ -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.table.sources
+
+/**
+  * Adds support for projection push-down to a [[TableSource]] with nested fields.
+  * A [[TableSource]] extending this interface is able
+  * to project the nested fields of the returned table.
+  *
+  * @tparam T The return type of the [[NestedFieldsProjectableTableSource]].
+  */
+trait NestedFieldsProjectableTableSource[T] {
+
+  /**
+    * Creates a copy of the [[TableSource]] that projects its output on the specified nested fields.
+    *
+    * @param fields The indexes of the fields to return.
+    * @param nestedFields The accessed nested fields of the fields to return.
+    *
+    * e.g.
+    * tableSchema = {
+    *       id,
+    *       student<\school<\city, tuition>, age, name>,
+    *       teacher<\age, name>
+    *       }
+    *
+    * select (id, student.school.city, student.age, teacher)
+    *
+    * fields = field = [0, 1, 2]
+    * nestedFields  \[\["*"], ["school.city", "age"], ["*"\]\]
+    *
+    * @return A copy of the [[TableSource]] that projects its output.
+    */
+  def projectNestedFields(
+      fields: Array[Int],
+      nestedFields: Array[Array[String]]): TableSource[T]
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5c37e55c/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala
index b0a5fcf..999d20f 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/util/RexProgramExtractorTest.scala
@@ -20,12 +20,15 @@ package org.apache.flink.table.plan.util
 
 import java.math.BigDecimal
 
-import org.apache.calcite.rex.{RexBuilder, RexProgramBuilder}
+import org.apache.calcite.rex.{RexBuilder, RexProgram, RexProgramBuilder}
 import org.apache.calcite.sql.SqlPostfixOperator
+import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, INTEGER, VARCHAR}
 import org.apache.calcite.sql.fun.SqlStdOperatorTable
 import org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.utils.InputTypeBuilder.inputOf
 import org.apache.flink.table.validate.FunctionCatalog
-import org.junit.Assert.{assertArrayEquals, assertEquals}
+import org.hamcrest.CoreMatchers.is
+import org.junit.Assert.{assertArrayEquals, assertEquals, assertThat}
 import org.junit.Test
 
 import scala.collection.JavaConverters._
@@ -306,6 +309,180 @@ class RexProgramExtractorTest extends RexProgramTestBase {
       unconvertedRexNodes(1).toString)
   }
 
+  @Test
+  def testExtractRefNestedInputFields(): Unit = {
+    val rexProgram = buildRexProgramWithNesting()
+
+    val usedFields = RexProgramExtractor.extractRefInputFields(rexProgram)
+    val usedNestedFields = RexProgramExtractor.extractRefNestedInputFields(rexProgram, usedFields)
+
+    val expected = Array(Array("amount"), Array("*"))
+    assertThat(usedNestedFields, is(expected))
+  }
+
+  @Test
+  def testExtractRefNestedInputFieldsWithNoNesting(): Unit = {
+    val rexProgram = buildSimpleRexProgram()
+
+    val usedFields = RexProgramExtractor.extractRefInputFields(rexProgram)
+    val usedNestedFields = RexProgramExtractor.extractRefNestedInputFields(rexProgram, usedFields)
+
+    val expected = Array(Array("*"), Array("*"), Array("*"))
+    assertThat(usedNestedFields, is(expected))
+  }
+
+  @Test
+  def testExtractDeepRefNestedInputFields(): Unit = {
+    val rexProgram = buildRexProgramWithDeepNesting()
+
+    val usedFields = RexProgramExtractor.extractRefInputFields(rexProgram)
+    val usedNestedFields = RexProgramExtractor.extractRefNestedInputFields(rexProgram, usedFields)
+
+    val expected = Array(
+      Array("amount"),
+      Array("*"),
+      Array("with.deeper.entry", "with.deep.entry"))
+
+    assertThat(usedFields, is(Array(1, 0, 2)))
+    assertThat(usedNestedFields, is(expected))
+  }
+
+  private def buildRexProgramWithDeepNesting(): RexProgram = {
+
+    // person input
+    val passportRow = inputOf(typeFactory)
+      .field("id", VARCHAR)
+      .field("status", VARCHAR)
+      .build
+
+    val personRow = inputOf(typeFactory)
+      .field("name", VARCHAR)
+      .field("age", INTEGER)
+      .nestedField("passport", passportRow)
+      .build
+
+    // payment input
+    val paymentRow = inputOf(typeFactory)
+      .field("id", BIGINT)
+      .field("amount", INTEGER)
+      .build
+
+    // deep field input
+    val deepRowType = inputOf(typeFactory)
+      .field("entry", VARCHAR)
+      .build
+
+    val entryRowType = inputOf(typeFactory)
+      .nestedField("inside", deepRowType)
+      .build
+
+    val deeperRowType = inputOf(typeFactory)
+      .nestedField("entry", entryRowType)
+      .build
+
+    val withRowType = inputOf(typeFactory)
+      .nestedField("deep", deepRowType)
+      .nestedField("deeper", deeperRowType)
+      .build
+
+    val fieldRowType = inputOf(typeFactory)
+      .nestedField("with", withRowType)
+      .build
+
+    // main input
+    val inputRowType = inputOf(typeFactory)
+      .nestedField("persons", personRow)
+      .nestedField("payments", paymentRow)
+      .nestedField("field", fieldRowType)
+      .build
+
+    // inputRowType
+    //
+    // [ persons:  [ name: VARCHAR, age:  INT, passport: [id: VARCHAR, status: VARCHAR ] ],
+    //   payments: [ id: BIGINT, amount: INT ],
+    //   field:    [ with: [ deep: [ entry: VARCHAR ],
+    //                       deeper: [ entry: [ inside: [entry: VARCHAR ] ] ]
+    //             ] ]
+    // ]
+
+    val builder = new RexProgramBuilder(inputRowType, rexBuilder)
+
+    val t0 = rexBuilder.makeInputRef(personRow, 0)
+    val t1 = rexBuilder.makeInputRef(paymentRow, 1)
+    val t2 = rexBuilder.makeInputRef(fieldRowType, 2)
+    val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(10L))
+
+    // person
+    val person$pass = rexBuilder.makeFieldAccess(t0, "passport", false)
+    val person$pass$stat = rexBuilder.makeFieldAccess(person$pass, "status", false)
+
+    // payment
+    val pay$amount = rexBuilder.makeFieldAccess(t1, "amount", false)
+    val multiplyAmount = builder.addExpr(
+      rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, pay$amount, t3))
+
+    // field
+    val field$with = rexBuilder.makeFieldAccess(t2, "with", false)
+    val field$with$deep = rexBuilder.makeFieldAccess(field$with, "deep", false)
+    val field$with$deeper = rexBuilder.makeFieldAccess(field$with, "deeper", false)
+    val field$with$deep$entry = rexBuilder.makeFieldAccess(field$with$deep, "entry", false)
+    val field$with$deeper$entry = rexBuilder.makeFieldAccess(field$with$deeper, "entry", false)
+    val field$with$deeper$entry$inside = rexBuilder
+      .makeFieldAccess(field$with$deeper$entry, "inside", false)
+    val field$with$deeper$entry$inside$entry = rexBuilder
+      .makeFieldAccess(field$with$deeper$entry$inside, "entry", false)
+
+    builder.addProject(multiplyAmount, "amount")
+    builder.addProject(person$pass$stat, "status")
+    builder.addProject(field$with$deep$entry, "entry")
+    builder.addProject(field$with$deeper$entry$inside$entry, "entry")
+    builder.addProject(field$with$deeper$entry, "entry2")
+    builder.addProject(t0, "person")
+
+    // Program
+    // (
+    //   payments.amount * 10),
+    //   persons.passport.status,
+    //   field.with.deep.entry
+    //   field.with.deeper.entry.inside.entry
+    //   field.with.deeper.entry
+    //   persons
+    // )
+
+    builder.getProgram
+
+  }
+
+  private def buildRexProgramWithNesting(): RexProgram = {
+
+    val personRow = inputOf(typeFactory)
+      .field("name", INTEGER)
+      .field("age", VARCHAR)
+      .build
+
+    val paymentRow = inputOf(typeFactory)
+      .field("id", BIGINT)
+      .field("amount", INTEGER)
+      .build
+
+    val types = List(personRow, paymentRow).asJava
+    val names = List("persons", "payments").asJava
+    val inputRowType = typeFactory.createStructType(types, names)
+
+    val builder = new RexProgramBuilder(inputRowType, rexBuilder)
+
+    val t0 = rexBuilder.makeInputRef(types.get(0), 0)
+    val t1 = rexBuilder.makeInputRef(types.get(1), 1)
+    val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L))
+
+    val payment$amount = rexBuilder.makeFieldAccess(t1, "amount", false)
+
+    builder.addProject(payment$amount, "amount")
+    builder.addProject(t0, "persons")
+    builder.addProject(t2, "number")
+    builder.getProgram
+  }
+
   private def testExtractSinglePostfixCondition(
       fieldIndex: Integer,
       op: SqlPostfixOperator,

http://git-wip-us.apache.org/repos/asf/flink/blob/5c37e55c/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InputTypeBuilder.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InputTypeBuilder.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InputTypeBuilder.scala
new file mode 100644
index 0000000..6f11f88
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InputTypeBuilder.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.table.utils
+
+import org.apache.calcite.adapter.java.JavaTypeFactory
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql.`type`.SqlTypeName
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+class InputTypeBuilder(typeFactory: JavaTypeFactory) {
+
+  private val names = mutable.ListBuffer[String]()
+  private val types = mutable.ListBuffer[RelDataType]()
+
+  def field(name: String, `type`: SqlTypeName): InputTypeBuilder = {
+    names += name
+    types += typeFactory.createSqlType(`type`)
+    this
+  }
+
+  def nestedField(name: String, `type`: RelDataType): InputTypeBuilder = {
+    names += name
+    types += `type`
+    this
+  }
+
+  def build: RelDataType = {
+    typeFactory.createStructType(types.asJava, names.asJava)
+  }
+}
+
+object InputTypeBuilder {
+
+  def inputOf(typeFactory: JavaTypeFactory) = new InputTypeBuilder(typeFactory)
+}


[19/50] [abbrv] flink git commit: [FLINK-6128] [tests] Optimize JVM options to improve test performance

Posted by fh...@apache.org.
[FLINK-6128] [tests] Optimize JVM options to improve test performance

Decrease the time to run TravisCI builds and reduce OOM by
- increasing the heap size
- switching to Java's serial garbage collector
- removing the UseGCOverheadLimit option

This closes #3571


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

Branch: refs/heads/table-retraction
Commit: 976e03c1ee0c1ac03e7755aef6a38ee0abc35b2f
Parents: 524b20f
Author: DmytroShkvyra <ds...@gmail.com>
Authored: Mon Mar 20 13:12:37 2017 +0200
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 24 12:20:18 2017 -0400

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/976e03c1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 17984d3..874bead 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1027,7 +1027,7 @@ under the License.
 						<forkNumber>0${surefire.forkNumber}</forkNumber>
 						<log4j.configuration>${log4j.configuration}</log4j.configuration>
 					</systemPropertyVariables>
-					<argLine>-Xms256m -Xmx800m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit</argLine>
+					<argLine>-Xms256m -Xmx1536m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseSerialGC</argLine>
 				</configuration>
 				<executions>
 					<!--execute all the unit tests-->


[33/50] [abbrv] flink git commit: [FLINK-6197] [cep] Add support for iterative conditions.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/OrCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/OrCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/OrCondition.java
new file mode 100644
index 0000000..6aaa4bb
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/OrCondition.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cep.pattern.conditions;
+
+/**
+ * A {@link IterativeCondition condition} which combines two conditions with a logical
+ * {@code OR} and returns {@code true} if at least one is {@code true}.
+ *
+ * @param <T> Type of the element to filter
+ */
+public class OrCondition<T> extends IterativeCondition<T> {
+
+	private static final long serialVersionUID = 2554610954278485106L;
+
+	private final IterativeCondition<T> left;
+	private final IterativeCondition<T> right;
+
+	public OrCondition(final IterativeCondition<T> left, final IterativeCondition<T> right) {
+		this.left = left;
+		this.right = right;
+	}
+
+	@Override
+	public boolean filter(T value, Context<T> ctx) throws Exception {
+		return left.filter(value, ctx) || right.filter(value, ctx);
+	}
+
+	/**
+	 * @return One of the {@link IterativeCondition conditions} combined in this condition.
+	 */
+	public IterativeCondition<T> getLeft() {
+		return left;
+	}
+
+	/**
+	 * @return One of the {@link IterativeCondition conditions} combined in this condition.
+	 */
+	public IterativeCondition<T> getRight() {
+		return right;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SimpleCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SimpleCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SimpleCondition.java
new file mode 100644
index 0000000..9ca52c5
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SimpleCondition.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cep.pattern.conditions;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+
+/**
+ *  A user-defined condition that decides if an element should be accepted in the pattern or not.
+ * Accepting an element also signals a state transition for the corresponding {@link org.apache.flink.cep.nfa.NFA}.
+ *
+ * <p>Contrary to the {@link IterativeCondition}, conditions that extend this class do not have access to the
+ * previously accepted elements in the pattern. Conditions that extend this class are simple {@code filter(...)}
+ * functions that decide based on the properties of the element at hand.
+ */
+public abstract class SimpleCondition<T> extends IterativeCondition<T> implements FilterFunction<T> {
+
+	private static final long serialVersionUID = 4942618239408140245L;
+
+	@Override
+	public boolean filter(T value, Context<T> ctx) throws Exception {
+		return filter(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SubtypeCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SubtypeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SubtypeCondition.java
new file mode 100644
index 0000000..91f6c21
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/SubtypeCondition.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cep.pattern.conditions;
+
+/**
+ * A {@link IterativeCondition condition} which filters elements of the given type.
+ * An element is filtered out iff it is not assignable to the given subtype of {@code T}.
+ *
+ * @param <T> Type of the elements to be filtered
+ */
+public class SubtypeCondition<T> extends SimpleCondition<T> {
+	private static final long serialVersionUID = -2990017519957561355L;
+
+	/** The subtype to filter for. */
+	private final Class<? extends T> subtype;
+
+	public SubtypeCondition(final Class<? extends T> subtype) {
+		this.subtype = subtype;
+	}
+
+	@Override
+	public boolean filter(T value) throws Exception {
+		return subtype.isAssignableFrom(value.getClass());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java
index 5887017..42117ee 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java
@@ -18,11 +18,11 @@
 
 package org.apache.flink.cep;
 
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -81,7 +81,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			new Event(8, "end", 1.0)
 		);
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -89,7 +89,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		})
 		.followedBy("middle").subtype(SubEvent.class).where(
-				new FilterFunction<SubEvent>() {
+				new SimpleCondition<SubEvent>() {
 
 					@Override
 					public boolean filter(SubEvent value) throws Exception {
@@ -97,7 +97,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 					}
 				}
 			)
-		.followedBy("end").where(new FilterFunction<Event>() {
+		.followedBy("end").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -156,7 +156,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -164,7 +164,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		})
 			.followedBy("middle").subtype(SubEvent.class).where(
-				new FilterFunction<SubEvent>() {
+				new SimpleCondition<SubEvent>() {
 
 					@Override
 					public boolean filter(SubEvent value) throws Exception {
@@ -172,7 +172,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 					}
 				}
 			)
-			.followedBy("end").where(new FilterFunction<Event>() {
+			.followedBy("end").where(new SimpleCondition<Event>() {
 
 				@Override
 				public boolean filter(Event value) throws Exception {
@@ -236,19 +236,19 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).followedBy("end").where(new FilterFunction<Event>() {
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -325,19 +325,19 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).followedBy("end").where(new FilterFunction<Event>() {
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -378,7 +378,7 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 
 		Pattern<Tuple2<Integer, Integer>, ?> pattern =
 			Pattern.<Tuple2<Integer, Integer>>begin("start")
-				.where(new FilterFunction<Tuple2<Integer, Integer>>() {
+				.where(new SimpleCondition<Tuple2<Integer, Integer>>() {
 					@Override
 					public boolean filter(Tuple2<Integer, Integer> rec) throws Exception {
 						return rec.f1 == 1;
@@ -456,19 +456,19 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).followedBy("end").where(new FilterFunction<Event>() {
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -524,26 +524,26 @@ public class CEPITCase extends StreamingMultipleProgramsTestBase {
 		);
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start")
-			.where(new FilterFunction<Event>() {
+			.where(new SimpleCondition<Event>() {
 				@Override
 				public boolean filter(Event value) throws Exception {
 					return value.getName().equals("start");
 				}
 			})
 			.followedBy("middle")
-			.where(new FilterFunction<Event>() {
+			.where(new SimpleCondition<Event>() {
 				@Override
 				public boolean filter(Event value) throws Exception {
 					return value.getPrice() == 2.0;
 				}
 			})
-			.or(new FilterFunction<Event>() {
+			.or(new SimpleCondition<Event>() {
 				@Override
 				public boolean filter(Event value) throws Exception {
 					return value.getPrice() == 5.0;
 				}
 			})
-			.followedBy("end").where(new FilterFunction<Event>() {
+			.followedBy("end").where(new SimpleCondition<Event>() {
 
 				@Override
 				public boolean filter(Event value) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 5b05f19..197767e 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -18,20 +18,26 @@
 
 package org.apache.flink.cep.nfa;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import org.apache.flink.api.common.functions.FilterFunction;
+import com.google.common.primitives.Doubles;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.TestLogger;
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -39,7 +45,6 @@ import java.util.Map;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class NFAITCase extends TestLogger {
 
@@ -58,23 +63,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<Event>(new Event(43, "start", 1.0), 4));
 		inputEvents.add(new StreamRecord<Event>(endEvent, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		})
-		.followedBy("middle").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
-				private static final long serialVersionUID = 6215754202506583964L;
+		}).followedBy("middle").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 6215754202506583964L;
 
-				@Override
-				public boolean filter(SubEvent value) throws Exception {
-					return value.getVolume() > 5.0;
-				}
-			})
-		.followedBy("end").where(new FilterFunction<Event>() {
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getVolume() > 5.0;
+			}
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 7056763917392056548L;
 
 			@Override
@@ -113,14 +116,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
 		inputEvents.add(new StreamRecord<>(end, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).next("end").where(new FilterFunction<Event>() {
+		}).next("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -163,14 +166,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
 		inputEvents.add(new StreamRecord<>(end, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).next("end").where(new FilterFunction<Event>() {
+		}).next("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -217,21 +220,21 @@ public class NFAITCase extends TestLogger {
 		events.add(new StreamRecord<Event>(new Event(6, "end", 1.0), 13));
 
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 7907391379273505897L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -3268741540234334074L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).followedBy("end").where(new FilterFunction<Event>() {
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -8995174172182138608L;
 
 			@Override
@@ -240,11 +243,12 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(10));
 
-
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
 		for (StreamRecord<Event> event: events) {
-			Collection<Map<String, Event>> patterns = nfa.process(event.getValue(), event.getTimestamp()).f0;
+			Collection<Map<String, Event>> patterns = nfa.process(
+					event.getValue(),
+					event.getTimestamp()).f0;
 
 			resultingPatterns.addAll(patterns);
 		}
@@ -269,7 +273,6 @@ public class NFAITCase extends TestLogger {
 		Set<Tuple2<Map<String, Event>, Long>> resultingTimeoutPatterns = new HashSet<>();
 		Set<Tuple2<Map<String, Event>, Long>> expectedTimeoutPatterns = new HashSet<>();
 
-
 		events.add(new StreamRecord<Event>(new Event(1, "start", 1.0), 1));
 		events.add(new StreamRecord<Event>(new Event(2, "start", 1.0), 2));
 		events.add(new StreamRecord<Event>(new Event(3, "middle", 1.0), 3));
@@ -296,21 +299,21 @@ public class NFAITCase extends TestLogger {
 		expectedTimeoutPatterns.add(Tuple2.of(timeoutPattern3, 11L));
 		expectedTimeoutPatterns.add(Tuple2.of(timeoutPattern4, 13L));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 7907391379273505897L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -3268741540234334074L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).followedBy("end").where(new FilterFunction<Event>() {
+		}).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = -8995174172182138608L;
 
 			@Override
@@ -319,7 +322,6 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(10));
 
-
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), true);
 
 		for (StreamRecord<Event> event: events) {
@@ -359,38 +361,35 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<Event>(nextOne2, 7));
 		inputEvents.add(new StreamRecord<Event>(endEvent, 8));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		})
-			.followedBy("middle-first").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
-				private static final long serialVersionUID = 6215754202506583964L;
+		}).followedBy("middle-first").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 6215754202506583964L;
 
-				@Override
-				public boolean filter(SubEvent value) throws Exception {
-					return value.getVolume() > 5.0;
-				}
-			})
-			.followedBy("middle-second").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
-				private static final long serialVersionUID = 6215754202506583964L;
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getVolume() > 5.0;
+			}
+		}).followedBy("middle-second").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 6215754202506583964L;
 
-				@Override
-				public boolean filter(SubEvent value) throws Exception {
-					return value.getName().equals("next-one");
-				}
-			})
-			.followedBy("end").where(new FilterFunction<Event>() {
-				private static final long serialVersionUID = 7056763917392056548L;
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getName().equals("next-one");
+			}
+		}).followedBy("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
 
-				@Override
-				public boolean filter(Event value) throws Exception {
-					return value.getName().equals("end");
-				}
-			});
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
@@ -443,44 +442,42 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(end3, 8));
 		inputEvents.add(new StreamRecord<>(end4, 9));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("b");
 			}
-		})
-			.followedBy("end2").where(new FilterFunction<Event>() {
-				private static final long serialVersionUID = 5726188262756267490L;
+		}).followedBy("end2").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
 
-				@Override
-				public boolean filter(Event value) throws Exception {
-					return value.getName().equals("d");
-				}
-			})
-			.followedBy("end3").where(new FilterFunction<Event>() {
-				private static final long serialVersionUID = 5726188262756267490L;
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).followedBy("end3").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
 
-				@Override
-				public boolean filter(Event value) throws Exception {
-					return value.getName().equals("e");
-				}
-			});
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("e");
+			}
+		});
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
@@ -533,21 +530,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -597,21 +594,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(true).followedBy("end1").where(new FilterFunction<Event>() {
+		}).zeroOrMore(true).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -660,14 +657,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 		inputEvents.add(new StreamRecord<>(end, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore().followedBy("end").where(new FilterFunction<Event>() {
+		}).zeroOrMore().followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -720,28 +717,28 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 		inputEvents.add(new StreamRecord<>(end, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle-first").where(new FilterFunction<Event>() {
+		}).followedBy("middle-first").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(false).followedBy("middle-second").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("middle-second").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("d");
 			}
-		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -799,35 +796,35 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(kleene2, 7));
 		inputEvents.add(new StreamRecord<>(end, 8));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("branching").where(new FilterFunction<Event>() {
+		}).followedBy("branching").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).followedBy("merging").where(new FilterFunction<Event>() {
+		}).followedBy("merging").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("f");
 			}
-		}).followedBy("kleene").where(new FilterFunction<Event>() {
+		}).followedBy("kleene").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("d");
 			}
-		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -881,14 +878,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
 		inputEvents.add(new StreamRecord<>(end, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("d");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -896,7 +893,7 @@ public class NFAITCase extends TestLogger {
 				return value.getName().equals("a");
 			}
 		}).zeroOrMore()
-			.next("end").where(new FilterFunction<Event>() {
+			.next("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -909,7 +906,6 @@ public class NFAITCase extends TestLogger {
 
 		Set<Set<Event>> resultingPatterns = new HashSet<>();
 
-
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, Event>> patterns = nfa.process(
 				inputEvent.getValue(),
@@ -937,29 +933,28 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 3));
 		inputEvents.add(new StreamRecord<>(end, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("d");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(false)
-			.next("end").where(new FilterFunction<Event>() {
-				private static final long serialVersionUID = 5726188262756267490L;
+		}).zeroOrMore(false).next("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
 
-				@Override
-				public boolean filter(Event value) throws Exception {
-					return value.getName().equals("b");
-				}
-			});
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
@@ -998,21 +993,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).oneOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+		}).oneOrMore(false).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1059,14 +1054,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(startEvent3, 5));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).oneOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+		}).oneOrMore(false).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1120,21 +1115,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5L));
 		inputEvents.add(new StreamRecord<>(endEvent, 6L));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 6215754202506583964L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("start");
 			}
-		}).next("middle").where(new FilterFunction<Event>() {
+		}).next("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 6215754202506583964L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("middle");
 			}
-		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 7056763917392056548L;
 
 			@Override
@@ -1181,21 +1176,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).oneOrMore(true).followedBy("end1").where(new FilterFunction<Event>() {
+		}).oneOrMore(true).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1240,21 +1235,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent, 5));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+		}).optional().followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1303,21 +1298,21 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).next("middle").where(new FilterFunction<Event>() {
+		}).next("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+		}).times(2).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1362,14 +1357,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+		}).times(2).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1411,14 +1406,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(startEvent, 1));
 		inputEvents.add(new StreamRecord<>(end1, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+		}).optional().followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1464,14 +1459,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1525,7 +1520,7 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(end2, 6));
 		inputEvents.add(new StreamRecord<>(end3, 6));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1571,14 +1566,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(startEvent, 1));
 		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1624,14 +1619,14 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
 		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1676,21 +1671,21 @@ public class NFAITCase extends TestLogger {
 		Event middleEvent3 = new Event(43, "a", 4.0);
 		Event end1 = new Event(44, "b", 5.0);
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).next("middle").where(new FilterFunction<Event>() {
+		}).next("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+		}).times(2).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1719,21 +1714,21 @@ public class NFAITCase extends TestLogger {
 		Event middleEvent = new Event(43, "a", 4.0);
 		Event end1 = new Event(44, "b", 5.0);
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+		}).optional().followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1761,21 +1756,21 @@ public class NFAITCase extends TestLogger {
 		Event middleEvent2 = new Event(42, "a", 3.0);
 		Event end1 = new Event(44, "b", 5.0);
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).oneOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+		}).oneOrMore(false).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1805,21 +1800,21 @@ public class NFAITCase extends TestLogger {
 		Event middleEvent2 = new Event(42, "a", 3.0);
 		Event end1 = new Event(44, "b", 5.0);
 
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("c");
 			}
-		}).followedBy("middle").where(new FilterFunction<Event>() {
+		}).followedBy("middle").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
 			public boolean filter(Event value) throws Exception {
 				return value.getName().equals("a");
 			}
-		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+		}).zeroOrMore(false).followedBy("end1").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
 			@Override
@@ -1841,4 +1836,459 @@ public class NFAITCase extends TestLogger {
 		assertEquals(true, nfa.isEmpty());
 	}
 
+
+	//////////////////////			Iterative BooleanConditions			/////////////////////////
+
+	private final Event startEvent1 = new Event(40, "start", 1.0);
+	private final Event startEvent2 = new Event(40, "start", 2.0);
+	private final Event startEvent3 = new Event(40, "start", 3.0);
+	private final Event startEvent4 = new Event(40, "start", 4.0);
+	private final SubEvent middleEvent1 = new SubEvent(41, "foo1", 1.0, 10);
+	private final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10);
+	private final SubEvent middleEvent3 = new SubEvent(43, "foo3", 3.0, 10);
+	private final SubEvent middleEvent4 = new SubEvent(43, "foo4", 1.0, 10);
+	private final Event nextOne = new Event(44, "next-one", 1.0);
+	private final Event endEvent = new Event(46, "end", 1.0);
+
+	@Test
+	public void testIterativeWithBranchingPatternEager() {
+		List<List<Event>> actual = testIterativeWithBranchingPattern(true);
+
+		compareMaps(actual,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent1),
+						Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4),
+						Lists.newArrayList(startEvent2, endEvent, middleEvent3)
+				)
+		);
+	}
+
+	@Test
+	public void testIterativeWithBranchingPatternCombinations() {
+		List<List<Event>> actual = testIterativeWithBranchingPattern(false);
+
+		compareMaps(actual,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, endEvent, middleEvent1, middleEvent2, middleEvent4),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent2, middleEvent1),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent4, middleEvent3),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent4, middleEvent2),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent3, middleEvent1),
+						Lists.newArrayList(startEvent2, endEvent, middleEvent3, middleEvent4),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent4, middleEvent1),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent4),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent1),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent2),
+						Lists.newArrayList(startEvent1, endEvent, middleEvent3),
+						Lists.newArrayList(startEvent2, endEvent, middleEvent4),
+						Lists.newArrayList(startEvent2, endEvent, middleEvent3)
+				)
+		);
+	}
+
+	private List<List<Event>> testIterativeWithBranchingPattern(boolean eager) {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1));
+		inputEvents.add(new StreamRecord<Event>(middleEvent1, 2));
+		inputEvents.add(new StreamRecord<Event>(middleEvent2, 3));
+		inputEvents.add(new StreamRecord<>(startEvent2, 4));
+		inputEvents.add(new StreamRecord<Event>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<Event>(middleEvent4, 5));
+		inputEvents.add(new StreamRecord<>(nextOne, 6));
+		inputEvents.add(new StreamRecord<>(endEvent, 8));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).followedBy("middle").subtype(SubEvent.class).where(new IterativeCondition<SubEvent>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(SubEvent value, Context<SubEvent> ctx) throws Exception {
+				if (!value.getName().startsWith("foo")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event event : ctx.getEventsForPattern("middle")) {
+					sum += event.getPrice();
+				}
+				sum += value.getPrice();
+				return Double.compare(sum, 5.0) < 0;
+			}
+		}).oneOrMore(eager).followedBy("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+					inputEvent.getValue(),
+					inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+
+		return resultingPatterns;
+	}
+
+	@Test
+	public void testIterativeWithLoopingStartingEager() {
+		List<List<Event>> actual = testIterativeWithLoopingStarting(true);
+
+		compareMaps(actual,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, startEvent2, endEvent),
+						Lists.newArrayList(startEvent1, endEvent),
+						Lists.newArrayList(startEvent2, endEvent),
+						Lists.newArrayList(startEvent3, endEvent),
+						Lists.newArrayList(endEvent)
+				)
+		);
+	}
+
+	@Test
+	public void testIterativeWithLoopingStartingCombination() {
+		List<List<Event>> actual = testIterativeWithLoopingStarting(false);
+
+		compareMaps(actual,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, startEvent2, endEvent),
+						Lists.newArrayList(startEvent1, startEvent3, endEvent),
+						Lists.newArrayList(startEvent1, endEvent),
+						Lists.newArrayList(startEvent2, endEvent),
+						Lists.newArrayList(startEvent3, endEvent),
+						Lists.newArrayList(endEvent)
+				)
+		);
+	}
+
+	private List<List<Event>> testIterativeWithLoopingStarting(boolean eager) {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
+		inputEvents.add(new StreamRecord<>(startEvent2, 2L));
+		inputEvents.add(new StreamRecord<>(startEvent3, 3L));
+		inputEvents.add(new StreamRecord<>(endEvent, 4L));
+
+		// for now, a pattern inherits its continuity property from the followedBy() or next(), and the default
+		// behavior (which is the one applied in the case that the pattern graph starts with such a pattern)
+		// of a looping pattern is with relaxed continuity (as in followedBy).
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new IterativeCondition<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+				if (!value.getName().equals("start")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event event : ctx.getEventsForPattern("start")) {
+					sum += event.getPrice();
+				}
+				sum += value.getPrice();
+				return Double.compare(sum, 5.0) < 0;
+			}
+		}).zeroOrMore(eager).followedBy("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+					inputEvent.getValue(),
+					inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+
+		return resultingPatterns;
+	}
+
+	@Test
+	public void testIterativeWithPrevPatternDependency() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
+		inputEvents.add(new StreamRecord<>(startEvent2, 2L));
+		inputEvents.add(new StreamRecord<>(endEvent, 4L));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).oneOrMore().followedBy("end").where(new IterativeCondition<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
+
+			@Override
+			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+				if (!value.getName().equals("end")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event event : ctx.getEventsForPattern("start")) {
+					sum += event.getPrice();
+				}
+				return Double.compare(sum, 2.0) >= 0;
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+					inputEvent.getValue(),
+					inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+
+		compareMaps(resultingPatterns,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, startEvent2, endEvent),
+						Lists.newArrayList(startEvent2, endEvent)
+				)
+		);
+	}
+
+	@Test
+	public void testIterativeWithABACPattern() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1L)); //1
+		inputEvents.add(new StreamRecord<Event>(middleEvent1, 2L)); //1
+
+		inputEvents.add(new StreamRecord<>(startEvent2, 2L)); //2
+		inputEvents.add(new StreamRecord<>(startEvent3, 2L)); //3
+		inputEvents.add(new StreamRecord<Event>(middleEvent2, 2L)); //2
+
+		inputEvents.add(new StreamRecord<>(startEvent4, 2L)); //4
+		inputEvents.add(new StreamRecord<Event>(middleEvent3, 2L)); //3
+		inputEvents.add(new StreamRecord<Event>(middleEvent4, 2L)); //1
+		inputEvents.add(new StreamRecord<>(endEvent, 4L));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).followedBy("middle1").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 2178338526904474690L;
+
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getName().startsWith("foo");
+			}
+		}).followedBy("middle2").where(new IterativeCondition<Event>() {
+			private static final long serialVersionUID = -1223388426808292695L;
+
+			@Override
+			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+				if (!value.getName().equals("start")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event e: ctx.getEventsForPattern("middle2")) {
+					sum += e.getPrice();
+				}
+				sum += value.getPrice();
+				return Double.compare(sum, 5.0) <= 0;
+			}
+		}).oneOrMore().followedBy("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 562590474115118323L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+					inputEvent.getValue(),
+					inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+
+		compareMaps(resultingPatterns,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent1, endEvent),
+						Lists.newArrayList(startEvent1, middleEvent1, startEvent2, endEvent),
+						Lists.newArrayList(startEvent1, middleEvent2, startEvent4, endEvent),
+						Lists.newArrayList(startEvent2, middleEvent2, startEvent4, endEvent),
+						Lists.newArrayList(startEvent3, middleEvent2, startEvent4, endEvent)
+				)
+		);
+	}
+
+	@Test
+	public void testIterativeWithPrevPatternDependencyAfterBranching() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
+		inputEvents.add(new StreamRecord<>(startEvent2, 2L));
+		inputEvents.add(new StreamRecord<Event>(middleEvent1, 4L));
+		inputEvents.add(new StreamRecord<>(startEvent3, 5L));
+		inputEvents.add(new StreamRecord<Event>(middleEvent2, 6L));
+		inputEvents.add(new StreamRecord<>(endEvent, 7L));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).oneOrMore().followedBy("middle1").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 2178338526904474690L;
+
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getName().startsWith("foo");
+			}
+		}).followedBy("end").where(new IterativeCondition<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
+
+			@Override
+			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+				if (!value.getName().equals("end")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event event : ctx.getEventsForPattern("start")) {
+					sum += event.getPrice();
+				}
+				return Double.compare(sum, 2.0) >= 0;
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		List<List<Event>> resultingPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+					inputEvent.getValue(),
+					inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> p: patterns) {
+				resultingPatterns.add(new ArrayList<>(p.values()));
+			}
+		}
+
+		compareMaps(resultingPatterns,
+				Lists.<List<Event>>newArrayList(
+						Lists.newArrayList(startEvent1, startEvent2, middleEvent1, endEvent),
+						Lists.newArrayList(startEvent2, middleEvent1, endEvent),
+						Lists.newArrayList(startEvent1, startEvent2, middleEvent2, endEvent),
+						Lists.newArrayList(startEvent1, startEvent2, startEvent3, middleEvent2, endEvent),
+						Lists.newArrayList(startEvent2, startEvent3, middleEvent2, endEvent),
+						Lists.newArrayList(startEvent2, middleEvent2, endEvent),
+						Lists.newArrayList(startEvent3, middleEvent2, endEvent)
+				)
+		);
+	}
+
+	private void compareMaps(List<List<Event>> actual, List<List<Event>> expected) {
+		Assert.assertEquals(expected.size(), actual.size());
+
+		for (List<Event> p: actual) {
+			Collections.sort(p, new EventComparator());
+		}
+
+		for (List<Event> p: expected) {
+			Collections.sort(p, new EventComparator());
+		}
+
+		Collections.sort(actual, new ListEventComparator());
+		Collections.sort(expected, new ListEventComparator());
+		Assert.assertArrayEquals(expected.toArray(), actual.toArray());
+	}
+
+	private class ListEventComparator implements Comparator<List<Event>> {
+
+		@Override
+		public int compare(List<Event> o1, List<Event> o2) {
+			int sizeComp = Integer.compare(o1.size(), o2.size());
+			if (sizeComp == 0) {
+				EventComparator comp = new EventComparator();
+				for (int i = 0; i < o1.size(); i++) {
+					int eventComp = comp.compare(o1.get(i), o2.get(i));
+					if (eventComp != 0) {
+						return eventComp;
+					}
+				}
+				return 0;
+			} else {
+				return sizeComp;
+			}
+		}
+	}
+
+	private class EventComparator implements Comparator<Event> {
+
+		@Override
+		public int compare(Event o1, Event o2) {
+			int nameComp = o1.getName().compareTo(o2.getName());
+			int priceComp = Doubles.compare(o1.getPrice(), o2.getPrice());
+			int idComp = Integer.compare(o1.getId(), o2.getId());
+			if (nameComp == 0) {
+				if (priceComp == 0) {
+					return idComp;
+				} else {
+					return priceComp;
+				}
+			} else {
+				return nameComp;
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
index 40a0e7e..d2e392b 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
@@ -19,9 +19,9 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.pattern.FilterFunctions;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -58,7 +58,7 @@ public class NFATest extends TestLogger {
 
 		startState.addTake(
 			endState,
-			new FilterFunction<Event>() {
+			new SimpleCondition<Event>() {
 				private static final long serialVersionUID = -4869589195918650396L;
 
 				@Override
@@ -68,7 +68,7 @@ public class NFATest extends TestLogger {
 			});
 		endState.addTake(
 			endingState,
-			new FilterFunction<Event>() {
+			new SimpleCondition<Event>() {
 				private static final long serialVersionUID = 2979804163709590673L;
 
 				@Override
@@ -76,7 +76,7 @@ public class NFATest extends TestLogger {
 					return value.getName().equals("end");
 				}
 			});
-		endState.addIgnore(FilterFunctions.<Event>trueFunction());
+		endState.addIgnore(BooleanConditions.<Event>trueFunction());
 
 		nfa.addState(startState);
 		nfa.addState(endState);
@@ -241,7 +241,7 @@ public class NFATest extends TestLogger {
 
 		startState.addTake(
 			endState,
-			new FilterFunction<Event>() {
+			new SimpleCondition<Event>() {
 				private static final long serialVersionUID = -4869589195918650396L;
 
 				@Override
@@ -251,7 +251,7 @@ public class NFATest extends TestLogger {
 			});
 		endState.addTake(
 			endingState,
-			new FilterFunction<Event>() {
+			new SimpleCondition<Event>() {
 				private static final long serialVersionUID = 2979804163709590673L;
 
 				@Override
@@ -259,7 +259,7 @@ public class NFATest extends TestLogger {
 					return value.getName().equals("end");
 				}
 			});
-		endState.addIgnore(FilterFunctions.<Event>trueFunction());
+		endState.addIgnore(BooleanConditions.<Event>trueFunction());
 
 		nfa.addState(startState);
 		nfa.addState(endState);
@@ -268,7 +268,7 @@ public class NFATest extends TestLogger {
 		return nfa;
 	}
 
-	private static class NameFilter implements FilterFunction<Event> {
+	private static class NameFilter extends SimpleCondition<Event> {
 
 		private static final long serialVersionUID = 7472112494752423802L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
index 25618d5..f0a25d2 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
@@ -84,11 +84,16 @@ public class SharedBufferTest extends TestLogger {
 		sharedBuffer.put("b", events[7], timestamp, "a[]", events[6], timestamp, DeweyNumber.fromString("1.1.0"));
 
 		Collection<LinkedHashMultimap<String, Event>> patterns3 = sharedBuffer.extractPatterns("b", events[7], timestamp, DeweyNumber.fromString("1.1.0"));
-		sharedBuffer.remove("b", events[7], timestamp);
+		sharedBuffer.release("b", events[7], timestamp);
 		Collection<LinkedHashMultimap<String, Event>> patterns4 = sharedBuffer.extractPatterns("b", events[7], timestamp, DeweyNumber.fromString("1.1.0"));
 		Collection<LinkedHashMultimap<String, Event>> patterns1 = sharedBuffer.extractPatterns("b", events[5], timestamp, DeweyNumber.fromString("2.0.0"));
 		Collection<LinkedHashMultimap<String, Event>> patterns2 = sharedBuffer.extractPatterns("b", events[5], timestamp, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.remove("b", events[5], timestamp);
+		sharedBuffer.release("b", events[5], timestamp);
+
+		assertEquals(1L, patterns3.size());
+		assertEquals(0L, patterns4.size());
+		assertEquals(1L, patterns1.size());
+		assertEquals(1L, patterns2.size());
 
 		assertTrue(sharedBuffer.isEmpty());
 		assertTrue(patterns4.isEmpty());

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
index 93d78cc..80b1bcb 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.cep.nfa.compiler;
 
 import com.google.common.collect.Sets;
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -32,6 +31,7 @@ import org.apache.flink.cep.nfa.StateTransition;
 import org.apache.flink.cep.nfa.StateTransitionAction;
 import org.apache.flink.cep.pattern.MalformedPatternException;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.util.TestLogger;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,7 +48,7 @@ import static org.junit.Assert.assertTrue;
 
 public class NFACompilerTest extends TestLogger {
 
-	private static final FilterFunction<Event> startFilter = new FilterFunction<Event>() {
+	private static final SimpleCondition<Event> startFilter = new SimpleCondition<Event>() {
 		private static final long serialVersionUID = 3314714776170474221L;
 
 		@Override
@@ -57,7 +57,7 @@ public class NFACompilerTest extends TestLogger {
 		}
 	};
 
-	private static final FilterFunction<Event> endFilter = new FilterFunction<Event>() {
+	private static final SimpleCondition<Event> endFilter = new SimpleCondition<Event>() {
 		private static final long serialVersionUID = 3990995859716364087L;
 
 		@Override
@@ -91,7 +91,7 @@ public class NFACompilerTest extends TestLogger {
 	 * A filter implementation to test invalid pattern specification with
 	 * duplicate pattern names. Check {@link #testNFACompilerUniquePatternName()}.
 	 */
-	private static class TestFilter implements FilterFunction<Event> {
+	private static class TestFilter extends SimpleCondition<Event> {
 
 		private static final long serialVersionUID = -3863103355752267133L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
index 5a3e623..b83eb3c 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.cep.operator;
 
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.ByteSerializer;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
@@ -29,6 +28,7 @@ import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -239,7 +239,7 @@ public class CEPMigration11to13Test {
 		}
 	}
 
-	private static class StartFilter implements FilterFunction<Event> {
+	private static class StartFilter extends SimpleCondition<Event> {
 		private static final long serialVersionUID = 5726188262756267490L;
 
 		@Override
@@ -248,7 +248,7 @@ public class CEPMigration11to13Test {
 		}
 	}
 
-	private static class MiddleFilter implements FilterFunction<SubEvent> {
+	private static class MiddleFilter extends SimpleCondition<SubEvent> {
 		private static final long serialVersionUID = 6215754202506583964L;
 
 		@Override
@@ -257,7 +257,7 @@ public class CEPMigration11to13Test {
 		}
 	}
 
-	private static class EndFilter implements FilterFunction<Event> {
+	private static class EndFilter extends SimpleCondition<Event> {
 		private static final long serialVersionUID = 7056763917392056548L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
index 65fa733..f230bbc 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
@@ -17,7 +17,6 @@
  */
 package org.apache.flink.cep.operator;
 
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -26,6 +25,7 @@ import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
@@ -448,7 +448,7 @@ public class CEPMigration12to13Test {
 		}
 	}
 
-	private static class StartFilter implements FilterFunction<Event> {
+	private static class StartFilter extends SimpleCondition<Event> {
 		private static final long serialVersionUID = 5726188262756267490L;
 
 		@Override
@@ -457,7 +457,7 @@ public class CEPMigration12to13Test {
 		}
 	}
 
-	private static class MiddleFilter implements FilterFunction<SubEvent> {
+	private static class MiddleFilter extends SimpleCondition<SubEvent> {
 		private static final long serialVersionUID = 6215754202506583964L;
 
 		@Override
@@ -466,7 +466,7 @@ public class CEPMigration12to13Test {
 		}
 	}
 
-	private static class EndFilter implements FilterFunction<Event> {
+	private static class EndFilter extends SimpleCondition<Event> {
 		private static final long serialVersionUID = 7056763917392056548L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
index a99db05..726c8b8 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.operator;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -30,6 +29,7 @@ import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.watermark.Watermark;
@@ -434,7 +434,7 @@ public class CEPOperatorTest extends TestLogger {
 
 		harness.close();
 	}
-
+	
 	private void verifyWatermark(Object outputObject, long timestamp) {
 		assertTrue(outputObject instanceof Watermark);
 		assertEquals(timestamp, ((Watermark) outputObject).getTimestamp());
@@ -512,7 +512,7 @@ public class CEPOperatorTest extends TestLogger {
 		@Override
 		public NFA<Event> createNFA() {
 
-			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 						private static final long serialVersionUID = 5726188262756267490L;
 
 						@Override
@@ -520,7 +520,7 @@ public class CEPOperatorTest extends TestLogger {
 							return value.getName().equals("start");
 						}
 					})
-					.followedBy("middle").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
+					.followedBy("middle").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
 						private static final long serialVersionUID = 6215754202506583964L;
 
 						@Override
@@ -528,7 +528,7 @@ public class CEPOperatorTest extends TestLogger {
 							return value.getVolume() > 5.0;
 						}
 					})
-					.followedBy("end").where(new FilterFunction<Event>() {
+					.followedBy("end").where(new SimpleCondition<Event>() {
 						private static final long serialVersionUID = 7056763917392056548L;
 
 						@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
index 399662a..2c86648 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.operator;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.cep.Event;
@@ -27,6 +26,7 @@ import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
@@ -371,7 +371,7 @@ public class CEPRescalingTest {
 		@Override
 		public NFA<Event> createNFA() {
 
-			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 				private static final long serialVersionUID = 5726188262756267490L;
 
 				@Override
@@ -379,7 +379,7 @@ public class CEPRescalingTest {
 					return value.getName().equals("start");
 				}
 			})
-				.followedBy("middle").subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
+				.followedBy("middle").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
 					private static final long serialVersionUID = 6215754202506583964L;
 
 					@Override
@@ -387,7 +387,7 @@ public class CEPRescalingTest {
 						return value.getVolume() > 5.0;
 					}
 				})
-				.followedBy("end").where(new FilterFunction<Event>() {
+				.followedBy("end").where(new SimpleCondition<Event>() {
 					private static final long serialVersionUID = 7056763917392056548L;
 
 					@Override


[05/50] [abbrv] flink git commit: [FLINK-3318] Add support for quantifiers to CEP's pattern API

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index ccae848..825ba957 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -46,9 +46,9 @@ public class NFAITCase extends TestLogger {
 	public void testSimplePatternNFA() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
-		Event startEvent = new Event(42, "start", 1.0);
+		Event startEvent = new Event(41, "start", 1.0);
 		SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0);
-		Event endEvent=  new Event(43, "end", 1.0);
+		Event endEvent = new Event(43, "end", 1.0);
 
 		inputEvents.add(new StreamRecord<Event>(startEvent, 1));
 		inputEvents.add(new StreamRecord<Event>(new Event(43, "foobar", 1.0), 2));
@@ -102,6 +102,99 @@ public class NFAITCase extends TestLogger {
 		assertEquals(endEvent, patternMap.get("end"));
 	}
 
+	@Test
+	public void testStrictContinuityWithResults() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event end = new Event(42, "b", 4.0);
+
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(end, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).next("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(1, allPatterns.size());
+		assertEquals(Sets.<Set<Event>>newHashSet(
+			Sets.newHashSet(middleEvent1, end)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testStrictContinuityNoResults() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "c", 3.0);
+		Event end = new Event(43, "b", 4.0);
+
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(end, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).next("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+			}
+		}
+
+		assertEquals(Sets.newHashSet(), resultingPatterns);
+	}
+
 	/**
 	 * Tests that the NFA successfully filters out expired elements with respect to the window
 	 * length
@@ -327,6 +420,1247 @@ public class NFAITCase extends TestLogger {
 		), patterns);
 	}
 
+	@Test
+	public void testComplexBranchingAfterKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+		Event end2 = new Event(45, "d", 6.0);
+		Event end3 = new Event(46, "d", 7.0);
+		Event end4 = new Event(47, "e", 8.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+		inputEvents.add(new StreamRecord<>(end2, 7));
+		inputEvents.add(new StreamRecord<>(end3, 8));
+		inputEvents.add(new StreamRecord<>(end4, 9));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		})
+			.followedBy("end2").where(new FilterFunction<Event>() {
+				private static final long serialVersionUID = 5726188262756267490L;
+
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("d");
+				}
+			})
+			.followedBy("end3").where(new FilterFunction<Event>() {
+				private static final long serialVersionUID = 5726188262756267490L;
+
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("e");
+				}
+			});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(16, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, middleEvent3, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent3, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent2, middleEvent3, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent1, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent2, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent3, end1, end2, end4),
+			Sets.newHashSet(startEvent, end1, end2, end4),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, middleEvent3, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent3, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent2, middleEvent3, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent1, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent2, end1, end3, end4),
+			Sets.newHashSet(startEvent, middleEvent3, end1, end3, end4),
+			Sets.newHashSet(startEvent, end1, end3, end4)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(4, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1),
+			Sets.newHashSet(startEvent, middleEvent1, end1),
+			Sets.newHashSet(startEvent, middleEvent2, end1),
+			Sets.newHashSet(startEvent, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testEagerKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(true).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(4, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, middleEvent3, end1),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1),
+			Sets.newHashSet(startEvent, middleEvent1, end1),
+			Sets.newHashSet(startEvent, end1)
+		), resultingPatterns);
+	}
+
+
+	@Test
+	public void testBeginWithKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event middleEvent1 = new Event(40, "a", 2.0);
+		Event middleEvent2 = new Event(41, "a", 3.0);
+		Event middleEvent3 = new Event(41, "a", 3.0);
+		Event end = new Event(42, "b", 4.0);
+
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore().followedBy("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(7, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(middleEvent1, middleEvent2, middleEvent3, end),
+			Sets.newHashSet(middleEvent1, middleEvent2, end),
+			Sets.newHashSet(middleEvent2, middleEvent3, end),
+			Sets.newHashSet(middleEvent1, end),
+			Sets.newHashSet(middleEvent2, end),
+			Sets.newHashSet(middleEvent3, end),
+			Sets.newHashSet(end)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testKleeneStarAfterKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "d", 3.0);
+		Event middleEvent3 = new Event(43, "d", 4.0);
+		Event end = new Event(44, "e", 4.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle-first").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(false).followedBy("middle-second").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("e");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(8, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, middleEvent3, end),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent3, end),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end),
+			Sets.newHashSet(startEvent, middleEvent2, middleEvent3, end),
+			Sets.newHashSet(startEvent, middleEvent3, end),
+			Sets.newHashSet(startEvent, middleEvent2, end),
+			Sets.newHashSet(startEvent, middleEvent1, end),
+			Sets.newHashSet(startEvent, end)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testKleeneStarAfterBranching() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event merging = new Event(42, "f", 3.0);
+		Event kleene1 = new Event(43, "d", 4.0);
+		Event kleene2 = new Event(44, "d", 4.0);
+		Event end = new Event(45, "e", 4.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(merging, 5));
+		inputEvents.add(new StreamRecord<>(kleene1, 6));
+		inputEvents.add(new StreamRecord<>(kleene2, 7));
+		inputEvents.add(new StreamRecord<>(end, 8));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
 
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("branching").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).followedBy("merging").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("f");
+			}
+		}).followedBy("kleene").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("e");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(8, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, merging, end),
+			Sets.newHashSet(startEvent, middleEvent1, merging, kleene1, end),
+			Sets.newHashSet(startEvent, middleEvent1, merging, kleene2, end),
+			Sets.newHashSet(startEvent, middleEvent1, merging, kleene1, kleene2, end),
+			Sets.newHashSet(startEvent, middleEvent2, merging, end),
+			Sets.newHashSet(startEvent, middleEvent2, merging, kleene1, end),
+			Sets.newHashSet(startEvent, middleEvent2, merging, kleene2, end),
+			Sets.newHashSet(startEvent, middleEvent2, merging, kleene1, kleene2, end)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testStrictContinuityNoResultsAfterKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event start = new Event(40, "d", 2.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 2.0);
+		Event middleEvent3 = new Event(43, "c", 3.0);
+		Event end = new Event(44, "b", 4.0);
+
+		inputEvents.add(new StreamRecord<>(start, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 2));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
+		inputEvents.add(new StreamRecord<>(end, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore()
+			.next("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+			}
+		}
+
+		assertEquals(Sets.newHashSet(), resultingPatterns);
+	}
+
+	@Test
+	public void testStrictContinuityResultsAfterKleeneStar() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event start = new Event(40, "d", 2.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 2.0);
+		Event end = new Event(43, "b", 4.0);
+
+		inputEvents.add(new StreamRecord<>(start, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 2));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 3));
+		inputEvents.add(new StreamRecord<>(end, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(false)
+			.next("end").where(new FilterFunction<Event>() {
+				private static final long serialVersionUID = 5726188262756267490L;
+
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("b");
+				}
+			});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(2, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(start, middleEvent1, middleEvent2, end),
+			Sets.newHashSet(start, middleEvent2, end)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testAtLeastOne() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(3, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1),
+			Sets.newHashSet(startEvent, middleEvent1, end1),
+			Sets.newHashSet(startEvent, middleEvent2, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testBeginWithAtLeastOne() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent1 = new Event(41, "a", 2.0);
+		Event startEvent2 = new Event(42, "a", 3.0);
+		Event startEvent3 = new Event(42, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 3));
+		inputEvents.add(new StreamRecord<>(startEvent2, 4));
+		inputEvents.add(new StreamRecord<>(startEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(7, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent1, startEvent2, startEvent3, end1),
+			Sets.newHashSet(startEvent1, startEvent2, end1),
+			Sets.newHashSet(startEvent1, startEvent3, end1),
+			Sets.newHashSet(startEvent2, startEvent3, end1),
+			Sets.newHashSet(startEvent1, end1),
+			Sets.newHashSet(startEvent2, end1),
+			Sets.newHashSet(startEvent3, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testNextZeroOrMore() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "start", 1.0);
+		Event middleEvent1 = new Event(40, "middle", 2.0);
+		Event middleEvent2 = new Event(40, "middle", 3.0);
+		Event middleEvent3 = new Event(40, "middle", 4.0);
+		Event endEvent = new Event(46, "end", 1.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1L));
+		inputEvents.add(new StreamRecord<>(new Event(1, "event", 1.0), 2L));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3L));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4L));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5L));
+		inputEvents.add(new StreamRecord<>(endEvent, 6L));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).next("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("middle");
+			}
+		}).zeroOrMore(false).followedBy("end").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 7056763917392056548L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(1, allPatterns.size());
+		assertEquals(Sets.<Set<Event>>newHashSet(
+			Sets.newHashSet(startEvent, endEvent)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testAtLeastOneEager() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore(true).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(3, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, middleEvent3, end1),
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1),
+			Sets.newHashSet(startEvent, middleEvent1, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testOptional() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(2, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent, middleEvent, end1),
+			Sets.newHashSet(startEvent, end1)
+		), resultingPatterns);
+	}
+
+
+	@Test
+	public void testTimes() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 2));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).next("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(1, allPatterns.size());
+		assertEquals(Sets.<Set<Event>>newHashSet(
+			Sets.newHashSet(startEvent, middleEvent1, middleEvent2, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testStartWithTimes() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(middleEvent1, 2));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 4));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(2, allPatterns.size());
+		assertEquals(Sets.<Set<Event>>newHashSet(
+			Sets.newHashSet(middleEvent1, middleEvent2, end1),
+			Sets.newHashSet(middleEvent2, middleEvent3, end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testStartWithOptional() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		});
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(2, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent,  end1),
+			Sets.newHashSet(end1)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testEndWithZeroOrMore() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore();
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(4, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent,  middleEvent1, middleEvent2, middleEvent3),
+			Sets.newHashSet(startEvent,  middleEvent1, middleEvent2),
+			Sets.newHashSet(startEvent,  middleEvent1),
+			Sets.newHashSet(startEvent)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testStartAndEndWithZeroOrMore() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "d", 5.0);
+		Event end2 = new Event(45, "d", 5.0);
+		Event end3 = new Event(46, "d", 5.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+		inputEvents.add(new StreamRecord<>(end2, 6));
+		inputEvents.add(new StreamRecord<>(end3, 6));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore();
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(6, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(middleEvent1,  middleEvent2, middleEvent3),
+			Sets.newHashSet(middleEvent1,  middleEvent2),
+			Sets.newHashSet(middleEvent1),
+			Sets.newHashSet(middleEvent2,  middleEvent3),
+			Sets.newHashSet(middleEvent2),
+			Sets.newHashSet(middleEvent3)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testEndWithOptional() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).optional();
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(2, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent,  middleEvent1),
+			Sets.newHashSet(startEvent)
+		), resultingPatterns);
+	}
+
+	@Test
+	public void testEndWithOneOrMore() {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore();
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		Set<Set<Event>> resultingPatterns = new HashSet<>();
+		List<Collection<Event>> allPatterns = new ArrayList<>();
+
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			Collection<Map<String, Event>> patterns = nfa.process(
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
+
+			for (Map<String, Event> foundPattern : patterns) {
+				resultingPatterns.add(new HashSet<>(foundPattern.values()));
+				allPatterns.add(foundPattern.values());
+			}
+		}
+
+		assertEquals(3, allPatterns.size());
+		assertEquals(Sets.newHashSet(
+			Sets.newHashSet(startEvent,  middleEvent1, middleEvent2, middleEvent3),
+			Sets.newHashSet(startEvent,  middleEvent1, middleEvent2),
+			Sets.newHashSet(startEvent,  middleEvent1)
+		), resultingPatterns);
+	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
index 9f65132..40a0e7e 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
@@ -21,6 +21,7 @@ package org.apache.flink.cep.nfa;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.cep.Event;
+import org.apache.flink.cep.pattern.FilterFunctions;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -51,12 +52,12 @@ public class NFATest extends TestLogger {
 		streamEvents.add(new StreamRecord<>(new Event(3, "start", 3.0), 3L));
 		streamEvents.add(new StreamRecord<>(new Event(4, "end", 4.0), 4L));
 
-		State<Event> startingState = new State<>("", State.StateType.Start);
-		State<Event> startState = new State<>("start", State.StateType.Normal);
-		State<Event> endState = new State<>("end", State.StateType.Final);
-		StateTransition<Event> starting2Start = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			startState,
+		State<Event> startState = new State<>("start", State.StateType.Start);
+		State<Event> endState = new State<>("end", State.StateType.Normal);
+		State<Event> endingState = new State<>("", State.StateType.Final);
+
+		startState.addTake(
+			endState,
 			new FilterFunction<Event>() {
 				private static final long serialVersionUID = -4869589195918650396L;
 
@@ -64,12 +65,9 @@ public class NFATest extends TestLogger {
 				public boolean filter(Event value) throws Exception {
 					return value.getName().equals("start");
 				}
-			}
-		);
-
-		StateTransition<Event> start2End = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			endState,
+			});
+		endState.addTake(
+			endingState,
 			new FilterFunction<Event>() {
 				private static final long serialVersionUID = 2979804163709590673L;
 
@@ -77,18 +75,12 @@ public class NFATest extends TestLogger {
 				public boolean filter(Event value) throws Exception {
 					return value.getName().equals("end");
 				}
-			}
-		);
-
-		StateTransition<Event> start2Start = new StateTransition<>(StateTransitionAction.IGNORE, startState, null);
-
-		startingState.addStateTransition(starting2Start);
-		startState.addStateTransition(start2End);
-		startState.addStateTransition(start2Start);
+			});
+		endState.addIgnore(FilterFunctions.<Event>trueFunction());
 
-		nfa.addState(startingState);
 		nfa.addState(startState);
 		nfa.addState(endState);
+		nfa.addState(endingState);
 
 		Set<Map<String, Event>> expectedPatterns = new HashSet<>();
 
@@ -196,8 +188,10 @@ public class NFATest extends TestLogger {
 	public <T> Collection<Map<String, T>> runNFA(NFA<T> nfa, List<StreamRecord<T>> inputs) {
 		Set<Map<String, T>> actualPatterns = new HashSet<>();
 
-		for (StreamRecord<T> streamEvent: inputs) {
-			Collection<Map<String, T>> matchedPatterns = nfa.process(streamEvent.getValue(), streamEvent.getTimestamp()).f0;
+		for (StreamRecord<T> streamEvent : inputs) {
+			Collection<Map<String, T>> matchedPatterns = nfa.process(
+				streamEvent.getValue(),
+				streamEvent.getTimestamp()).f0;
 
 			actualPatterns.addAll(matchedPatterns);
 		}
@@ -213,24 +207,12 @@ public class NFATest extends TestLogger {
 		State<Event> startState = new State<>("start", State.StateType.Normal);
 		State<Event> endState = new State<>("end", State.StateType.Final);
 
-		StateTransition<Event> starting2Start = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			startState,
-			new NameFilter("start"));
 
-		StateTransition<Event> start2End = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			endState,
+		startingState.addTake(
+			new NameFilter("start"));
+		startState.addTake(
 			new NameFilter("end"));
-
-		StateTransition<Event> start2Start = new StateTransition<>(
-			StateTransitionAction.IGNORE,
-			startState,
-			null);
-
-		startingState.addStateTransition(starting2Start);
-		startState.addStateTransition(start2End);
-		startState.addStateTransition(start2Start);
+		startState.addIgnore(null);
 
 		nfa.addState(startingState);
 		nfa.addState(startState);
@@ -253,12 +235,12 @@ public class NFATest extends TestLogger {
 	private NFA<Event> createStartEndNFA(long windowLength) {
 		NFA<Event> nfa = new NFA<>(Event.createTypeSerializer(), windowLength, false);
 
-		State<Event> startingState = new State<>("", State.StateType.Start);
-		State<Event> startState = new State<>("start", State.StateType.Normal);
-		State<Event> endState = new State<>("end", State.StateType.Final);
-		StateTransition<Event> starting2Start = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			startState,
+		State<Event> startState = new State<>("start", State.StateType.Start);
+		State<Event> endState = new State<>("end", State.StateType.Normal);
+		State<Event> endingState = new State<>("", State.StateType.Final);
+
+		startState.addTake(
+			endState,
 			new FilterFunction<Event>() {
 				private static final long serialVersionUID = -4869589195918650396L;
 
@@ -267,10 +249,8 @@ public class NFATest extends TestLogger {
 					return value.getName().equals("start");
 				}
 			});
-
-		StateTransition<Event> start2End = new StateTransition<>(
-			StateTransitionAction.TAKE,
-			endState,
+		endState.addTake(
+			endingState,
 			new FilterFunction<Event>() {
 				private static final long serialVersionUID = 2979804163709590673L;
 
@@ -279,19 +259,11 @@ public class NFATest extends TestLogger {
 					return value.getName().equals("end");
 				}
 			});
+		endState.addIgnore(FilterFunctions.<Event>trueFunction());
 
-		StateTransition<Event> start2Start = new StateTransition<>(
-			StateTransitionAction.IGNORE,
-			startState,
-			null);
-
-		startingState.addStateTransition(starting2Start);
-		startState.addStateTransition(start2End);
-		startState.addStateTransition(start2Start);
-
-		nfa.addState(startingState);
 		nfa.addState(startState);
 		nfa.addState(endState);
+		nfa.addState(endingState);
 
 		return nfa;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
index d11f3a8..93d78cc 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
@@ -18,9 +18,11 @@
 
 package org.apache.flink.cep.nfa.compiler;
 
+import com.google.common.collect.Sets;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
@@ -28,22 +30,45 @@ import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.State;
 import org.apache.flink.cep.nfa.StateTransition;
 import org.apache.flink.cep.nfa.StateTransitionAction;
+import org.apache.flink.cep.pattern.MalformedPatternException;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.util.TestLogger;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
+import static com.google.common.collect.Sets.newHashSet;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class NFACompilerTest extends TestLogger {
 
+	private static final FilterFunction<Event> startFilter = new FilterFunction<Event>() {
+		private static final long serialVersionUID = 3314714776170474221L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getPrice() > 2;
+		}
+	};
+
+	private static final FilterFunction<Event> endFilter = new FilterFunction<Event>() {
+		private static final long serialVersionUID = 3990995859716364087L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("end");
+		}
+	};
+
+	private static final TypeSerializer<Event> serializer = TypeExtractor.createTypeInfo(Event.class)
+		.createSerializer(new ExecutionConfig());
+
 	@Rule
 	public ExpectedException expectedException = ExpectedException.none();
 
@@ -81,83 +106,96 @@ public class NFACompilerTest extends TestLogger {
 	 */
 	@Test
 	public void testNFACompilerWithSimplePattern() {
-		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
-			private static final long serialVersionUID = 3314714776170474221L;
-
-			@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getPrice() > 2;
-			}
-		})
-		.followedBy("middle").subtype(SubEvent.class)
-		.next("end").where(new FilterFunction<Event>() {
-				private static final long serialVersionUID = 3990995859716364087L;
+		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").where(startFilter)
+			.followedBy("middle").subtype(SubEvent.class)
+			.next("end").where(endFilter);
 
-				@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getName().equals("end");
-			}
-		});
-
-		TypeInformation<Event> typeInformation = TypeExtractor.createTypeInfo(Event.class);
-
-		NFA<Event> nfa = NFACompiler.compile(pattern, typeInformation.createSerializer(new ExecutionConfig()), false);
+		NFA<Event> nfa = NFACompiler.compile(pattern, serializer, false);
 
 		Set<State<Event>> states = nfa.getStates();
-
 		assertEquals(4, states.size());
 
 		Map<String, State<Event>> stateMap = new HashMap<>();
-
-		for (State<Event> state: states) {
+		for (State<Event> state : states) {
 			stateMap.put(state.getName(), state);
 		}
 
-		assertTrue(stateMap.containsKey(NFACompiler.BEGINNING_STATE_NAME));
-		State<Event> beginningState = stateMap.get(NFACompiler.BEGINNING_STATE_NAME);
-
-		assertTrue(beginningState.isStart());
-
 		assertTrue(stateMap.containsKey("start"));
 		State<Event> startState = stateMap.get("start");
+		assertTrue(startState.isStart());
+		final Set<Tuple2<String, StateTransitionAction>> startTransitions = unfoldTransitions(startState);
+		assertEquals(newHashSet(
+			Tuple2.of("middle", StateTransitionAction.TAKE)
+		), startTransitions);
 
-		Collection<StateTransition<Event>> startTransitions = startState.getStateTransitions();
-		Map<String, StateTransition<Event>> startTransitionMap = new HashMap<>();
+		assertTrue(stateMap.containsKey("middle"));
+		State<Event> middleState = stateMap.get("middle");
+		final Set<Tuple2<String, StateTransitionAction>> middleTransitions = unfoldTransitions(middleState);
+		assertEquals(newHashSet(
+			Tuple2.of("middle", StateTransitionAction.IGNORE),
+			Tuple2.of("end", StateTransitionAction.TAKE)
+		), middleTransitions);
 
-		for (StateTransition<Event> transition: startTransitions) {
-			startTransitionMap.put(transition.getTargetState().getName(), transition);
-		}
+		assertTrue(stateMap.containsKey("end"));
+		State<Event> endState = stateMap.get("end");
+		final Set<Tuple2<String, StateTransitionAction>> endTransitions = unfoldTransitions(endState);
+		assertEquals(newHashSet(
+			Tuple2.of(NFACompiler.ENDING_STATE_NAME, StateTransitionAction.TAKE)
+		), endTransitions);
+
+		assertTrue(stateMap.containsKey(NFACompiler.ENDING_STATE_NAME));
+		State<Event> endingState = stateMap.get(NFACompiler.ENDING_STATE_NAME);
+		assertTrue(endingState.isFinal());
+		assertEquals(0, endingState.getStateTransitions().size());
+	}
 
-		assertEquals(2, startTransitionMap.size());
-		assertTrue(startTransitionMap.containsKey("start"));
+	@Test
+	public void testNFACompilerWithKleeneStar() {
 
-		StateTransition<Event> reflexiveTransition = startTransitionMap.get("start");
-		assertEquals(StateTransitionAction.IGNORE, reflexiveTransition.getAction());
+		Pattern<Event, Event> pattern = Pattern.<Event>begin("start").where(startFilter)
+			.followedBy("middle").subtype(SubEvent.class).zeroOrMore()
+			.followedBy("end").where(endFilter);
 
-		assertTrue(startTransitionMap.containsKey("middle"));
-		StateTransition<Event> startMiddleTransition = startTransitionMap.get("middle");
-		assertEquals(StateTransitionAction.TAKE, startMiddleTransition.getAction());
+		NFA<Event> nfa = NFACompiler.compile(pattern, serializer, false);
 
-		assertTrue(stateMap.containsKey("middle"));
-		State<Event> middleState = stateMap.get("middle");
+		Set<State<Event>> states = nfa.getStates();
+		assertEquals(5, states.size());
 
-		Map<String, StateTransition<Event>> middleTransitionMap = new HashMap<>();
 
-		for (StateTransition<Event> transition: middleState.getStateTransitions()) {
-			middleTransitionMap.put(transition.getTargetState().getName(), transition);
+		Set<Tuple2<String, Set<Tuple2<String, StateTransitionAction>>>> stateMap = new HashSet<>();
+		for (State<Event> state : states) {
+			stateMap.add(Tuple2.of(state.getName(), unfoldTransitions(state)));
 		}
 
-		assertEquals(1, middleTransitionMap.size());
+		assertEquals(stateMap, newHashSet(
+			Tuple2.of("start", newHashSet(Tuple2.of("middle", StateTransitionAction.TAKE))),
+			Tuple2.of("middle", newHashSet(
+				Tuple2.of("middle", StateTransitionAction.IGNORE),
+				Tuple2.of("middle", StateTransitionAction.TAKE)
+			)),
+		    Tuple2.of("middle", newHashSet(
+			    Tuple2.of("middle", StateTransitionAction.IGNORE),
+			    Tuple2.of("middle", StateTransitionAction.TAKE),
+			    Tuple2.of("end", StateTransitionAction.PROCEED)
+		    )),
+			Tuple2.of("end", newHashSet(
+				Tuple2.of(NFACompiler.ENDING_STATE_NAME, StateTransitionAction.TAKE),
+				Tuple2.of("end", StateTransitionAction.IGNORE)
+			)),
+		    Tuple2.of(NFACompiler.ENDING_STATE_NAME, Sets.newHashSet())
+		));
 
-		assertTrue(middleTransitionMap.containsKey("end"));
-		StateTransition<Event> middleEndTransition = middleTransitionMap.get("end");
+	}
 
-		assertEquals(StateTransitionAction.TAKE, middleEndTransition.getAction());
 
-		assertTrue(stateMap.containsKey("end"));
-		State<Event> endState = stateMap.get("end");
-
-		assertTrue(endState.isFinal());
-		assertEquals(0, endState.getStateTransitions().size());
+	private <T> Set<Tuple2<String, StateTransitionAction>> unfoldTransitions(final State<T> state) {
+		final Set<Tuple2<String, StateTransitionAction>> transitions = new HashSet<>();
+		for (StateTransition<T> transition : state.getStateTransitions()) {
+			transitions.add(Tuple2.of(
+				transition.getTargetState().getName(),
+				transition.getAction()));
+		}
+		return transitions;
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9001c4ef/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
index 98c3f5a..68b0419 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/pattern/PatternTest.java
@@ -184,4 +184,58 @@ public class PatternTest extends TestLogger {
 		assertEquals(previous2.getName(), "start");
 	}
 
+	@Test(expected = MalformedPatternException.class)
+	public void testPatternCanHaveQuantifierSpecifiedOnce1() throws Exception {
+
+		Pattern.begin("start").where(new FilterFunction<Object>() {
+			@Override
+			public boolean filter(Object value) throws Exception {
+				return true;
+			}
+		}).oneOrMore().zeroOrMore();
+	}
+
+	@Test(expected = MalformedPatternException.class)
+	public void testPatternCanHaveQuantifierSpecifiedOnce2() throws Exception {
+
+		Pattern.begin("start").where(new FilterFunction<Object>() {
+			@Override
+			public boolean filter(Object value) throws Exception {
+				return true;
+			}
+		}).zeroOrMore().times(1);
+	}
+
+	@Test(expected = MalformedPatternException.class)
+	public void testPatternCanHaveQuantifierSpecifiedOnce3() throws Exception {
+
+		Pattern.begin("start").where(new FilterFunction<Object>() {
+			@Override
+			public boolean filter(Object value) throws Exception {
+				return true;
+			}
+		}).times(1).oneOrMore();
+	}
+
+	@Test(expected = MalformedPatternException.class)
+	public void testPatternCanHaveQuantifierSpecifiedOnce4() throws Exception {
+
+		Pattern.begin("start").where(new FilterFunction<Object>() {
+			@Override
+			public boolean filter(Object value) throws Exception {
+				return true;
+			}
+		}).oneOrMore().oneOrMore(true);
+	}
+
+	@Test(expected = MalformedPatternException.class)
+	public void testPatternCanHaveQuantifierSpecifiedOnce5() throws Exception {
+
+		Pattern.begin("start").where(new FilterFunction<Object>() {
+			@Override
+			public boolean filter(Object value) throws Exception {
+				return true;
+			}
+		}).oneOrMore(true).zeroOrMore(true);
+	}
 }


[39/50] [abbrv] flink git commit: [FLINK-6211] [kinesis] Fix AT_TIMESTAMP config validation for FlinkKinesisConsumer

Posted by fh...@apache.org.
[FLINK-6211] [kinesis] Fix AT_TIMESTAMP config validation for FlinkKinesisConsumer

This closes #3636.
This closes #3637.


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

Branch: refs/heads/table-retraction
Commit: 69843fefca49a9ce43db7aa4810716de72cc6b5f
Parents: cd55274
Author: Tony Wei <to...@gmail.com>
Authored: Wed Mar 29 15:08:04 2017 +0800
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Mar 29 22:24:44 2017 +0800

----------------------------------------------------------------------
 .../kinesis/util/KinesisConfigUtil.java         | 13 ++--
 .../kinesis/FlinkKinesisConsumerTest.java       | 65 ++++++++++++++++++++
 2 files changed, 73 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/69843fef/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
index eb29d78..59b8529 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
@@ -226,12 +226,15 @@ public class KinesisConfigUtil {
 		if (config.containsKey(key)) {
 			try {
 				initTimestampDateFormat.parse(config.getProperty(key));
-				double value = Double.parseDouble(config.getProperty(key));
-				if (value < 0) {
-					throw new NumberFormatException();
+			} catch (ParseException parseException) {
+				try {
+					double value = Double.parseDouble(config.getProperty(key));
+					if (value < 0) {
+						throw new NumberFormatException();
+					}
+				} catch (NumberFormatException numberFormatException){
+					throw new IllegalArgumentException(message);
 				}
-			} catch (ParseException | NumberFormatException e) {
-				throw new IllegalArgumentException(message);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/69843fef/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
index 2cc0270..45eb1bd 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
@@ -40,12 +40,14 @@ import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
+import java.text.ParseException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 import java.util.UUID;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Suite of FlinkKinesisConsumer tests for the methods called throughout the source life cycle.
@@ -165,6 +167,69 @@ public class FlinkKinesisConsumerTest {
 	}
 
 	@Test
+	public void testIllegalValueForInitialTimestampInConfig() {
+		exception.expect(IllegalArgumentException.class);
+		exception.expectMessage("Invalid value given for initial timestamp for AT_TIMESTAMP initial position in stream.");
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "AT_TIMESTAMP");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP, "-1.0");
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+	}
+
+	@Test
+	public void testDateStringInValidateOptionDatePropertyForInitialTimestampInConfig() {
+		String timestamp = "2016-04-04T19:58:46.480-00:00";
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "AT_TIMESTAMP");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP, timestamp);
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+
+		try {
+			KinesisConfigUtil.initTimestampDateFormat.parse(timestamp);
+		} catch (ParseException e){
+			e.printStackTrace();
+			fail();
+		}
+	}
+
+	@Test
+	public void testUnixTimestampInValidateOptionDatePropertyForInitialTimestampInConfig() {
+		String unixTimestamp = "1459799926.480";
+
+		Properties testConfig = new Properties();
+		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION, "AT_TIMESTAMP");
+		testConfig.setProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP, unixTimestamp);
+
+		KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+
+		try{
+			double value = Double.parseDouble(unixTimestamp);
+			if (value < 0) {
+				throw new NumberFormatException();
+			}
+		} catch (Exception e){
+			e.printStackTrace();
+			fail();
+		}
+	}
+
+	@Test
 	public void testUnparsableLongForDescribeStreamBackoffBaseMillisInConfig() {
 		exception.expect(IllegalArgumentException.class);
 		exception.expectMessage("Invalid value given for describe stream operation base backoff milliseconds");


[34/50] [abbrv] flink git commit: [FLINK-6197] [cep] Add support for iterative conditions.

Posted by fh...@apache.org.
[FLINK-6197] [cep] Add support for iterative conditions.

So far, the where clause only supported simple FilterFunction
conditions. With this, we add support for conditions where an
event is accepted not only based on its own properties, e.g.
name, as it was before, but also based on some statistic
computed over previously accepted events in the pattern, e.g.
if the price is higher than the average of the prices of the
previously accepted events.


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

Branch: refs/heads/table-retraction
Commit: 7fbdc100e19d69d7e31544c20fa94cb2b314ec12
Parents: f2a8bc9
Author: kl0u <kk...@gmail.com>
Authored: Wed Mar 22 15:52:07 2017 +0100
Committer: kl0u <kk...@gmail.com>
Committed: Mon Mar 27 20:11:14 2017 +0200

----------------------------------------------------------------------
 docs/dev/libs/cep.md                            |  94 ++-
 .../flink/cep/scala/pattern/Pattern.scala       |  30 +-
 .../flink/cep/scala/pattern/PatternTest.scala   |  44 +-
 .../apache/flink/cep/nfa/ComputationState.java  |  81 ++-
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 251 ++++---
 .../org/apache/flink/cep/nfa/SharedBuffer.java  | 118 +--
 .../java/org/apache/flink/cep/nfa/State.java    |  18 +-
 .../apache/flink/cep/nfa/StateTransition.java   |  51 +-
 .../flink/cep/nfa/compiler/NFACompiler.java     |  47 +-
 .../flink/cep/pattern/AndFilterFunction.java    |   6 +-
 .../flink/cep/pattern/FilterFunctions.java      |  44 --
 .../flink/cep/pattern/NotFilterFunction.java    |  42 --
 .../flink/cep/pattern/OrFilterFunction.java     |   6 +-
 .../org/apache/flink/cep/pattern/Pattern.java   |  44 +-
 .../cep/pattern/SubtypeFilterFunction.java      |   6 +-
 .../cep/pattern/conditions/AndCondition.java    |  57 ++
 .../pattern/conditions/BooleanConditions.java   |  53 ++
 .../pattern/conditions/IterativeCondition.java  |  98 +++
 .../cep/pattern/conditions/NotCondition.java    |  40 +
 .../cep/pattern/conditions/OrCondition.java     |  57 ++
 .../cep/pattern/conditions/SimpleCondition.java |  39 +
 .../pattern/conditions/SubtypeCondition.java    |  41 ++
 .../java/org/apache/flink/cep/CEPITCase.java    |  42 +-
 .../org/apache/flink/cep/nfa/NFAITCase.java     | 726 +++++++++++++++----
 .../java/org/apache/flink/cep/nfa/NFATest.java  |  18 +-
 .../apache/flink/cep/nfa/SharedBufferTest.java  |   9 +-
 .../flink/cep/nfa/compiler/NFACompilerTest.java |   8 +-
 .../cep/operator/CEPMigration11to13Test.java    |   8 +-
 .../cep/operator/CEPMigration12to13Test.java    |   8 +-
 .../flink/cep/operator/CEPOperatorTest.java     |  10 +-
 .../flink/cep/operator/CEPRescalingTest.java    |   8 +-
 .../apache/flink/cep/pattern/PatternTest.java   |  52 +-
 32 files changed, 1561 insertions(+), 595 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/docs/dev/libs/cep.md
----------------------------------------------------------------------
diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md
index 22cffbc..9d4ca91 100644
--- a/docs/dev/libs/cep.md
+++ b/docs/dev/libs/cep.md
@@ -124,13 +124,70 @@ val start : Pattern[Event, _] = Pattern.begin("start")
 </div>
 </div>
 
-Each state must have an unique name to identify the matched events later on.
+Each state must have a unique name to identify the matched events later on.
 Additionally, we can specify a filter condition for the event to be accepted as the start event via the `where` method.
+These filtering conditions can be either an `IterativeCondition` or a `SimpleCondition`. 
+
+**Iterative Conditions:** This type of conditions can iterate over the previously accepted elements in the pattern and 
+decide to accept a new element or not, based on some statistic over those elements. 
+
+Below is the code for an iterative condition that accepts elements whose name start with "foo" and for which, the sum 
+of the prices of the previously accepted elements for a state named "middle", plus the price of the current event, do 
+not exceed the value of 5.0. Iterative condition can be very powerful, especially in combination with quantifiers, e.g.
+`oneToMany` or `zeroToMany`.
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+start.where(new IterativeCondition<SubEvent>() {
+    @Override
+    public boolean filter(SubEvent value, Context<SubEvent> ctx) throws Exception {
+        if (!value.getName().startsWith("foo")) {
+            return false;
+        }
+        
+        double sum = 0.0;
+        for (Event event : ctx.getEventsForPattern("middle")) {
+            sum += event.getPrice();
+        }
+        sum += value.getPrice();
+        return Double.compare(sum, 5.0) < 0;
+    }
+});
+{% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+start.where(
+    (value, ctx) => {
+        var res = value.getName.startsWith("foo")
+        if (res) {
+            var sum = 0.0
+            for (e: Event <- ctx.getEventsForPattern("middle")) {
+                sum += e.getPrice
+            }
+            sum += value.getPrice
+            res = res && sum < 5.0
+        }
+        res
+    }
+)
+{% endhighlight %}
+</div>
+</div>
+
+<span class="label label-danger">Attention</span> The call to `Context.getEventsForPattern(...)` has to find the 
+elements that belong to the pattern. The cost of this operation can vary, so when implementing your condition, try 
+to minimize the times the method is called.
+
+**Simple Conditions:** This type of conditions extend the aforementioned `IterativeCondition` class. They are simple 
+filtering conditions that decide to accept an element or not, based only on properties of the element itself.
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-start.where(new FilterFunction<Event>() {
+start.where(new SimpleCondition<Event>() {
     @Override
     public boolean filter(Event value) {
         return ... // some condition
@@ -151,7 +208,7 @@ We can also restrict the type of the accepted event to some subtype of the initi
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-start.subtype(SubEvent.class).where(new FilterFunction<SubEvent>() {
+start.subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
     @Override
     public boolean filter(SubEvent value) {
         return ... // some condition
@@ -168,7 +225,7 @@ start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */)
 </div>
 
 As it can be seen here, the subtype condition can also be combined with an additional filter condition on the subtype.
-In fact you can always provide multiple conditions by calling `where` and `subtype` multiple times.
+In fact, you can always provide multiple conditions by calling `where` and `subtype` multiple times.
 These conditions will then be combined using the logical AND operator.
 
 In order to construct or conditions, one has to call the `or` method with a respective filter function.
@@ -177,12 +234,12 @@ Any existing filter function is then ORed with the given one.
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 {% highlight java %}
-pattern.where(new FilterFunction<Event>() {
+pattern.where(new SimpleCondition<Event>() {
     @Override
     public boolean filter(Event value) {
         return ... // some condition
     }
-}).or(new FilterFunction<Event>() {
+}).or(new SimpleCondition<Event>() {
     @Override
     public boolean filter(Event value) {
         return ... // or condition
@@ -201,8 +258,8 @@ pattern.where(event => ... /* some condition */).or(event => ... /* or condition
 Next, we can append further states to detect complex patterns.
 We can control the contiguity of two succeeding events to be accepted by the pattern.
 
-Strict contiguity means that two matching events have to succeed directly.
-This means that no other events can occur in between.
+Strict contiguity means that two matching events have to be directly the one after the other.
+This means that no other events can occur in between. 
 A strict contiguity pattern state can be created via the `next` method.
 
 <div class="codetabs" markdown="1">
@@ -236,7 +293,8 @@ val nonStrictNext : Pattern[Event, _] = start.followedBy("middle")
 </div>
 </div>
 It is also possible to define a temporal constraint for the pattern to be valid.
-For example, one can define that a pattern should occur within 10 seconds via the `within` method.
+For example, one can define that a pattern should occur within 10 seconds via the `within` method. 
+Temporal patterns are supported for both [processing and event time]({{site.baseurl}}/dev/event_time.html).
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -294,11 +352,11 @@ Pattern<Event, ?> followedBy = start.followedBy("next");
         <tr>
             <td><strong>Where</strong></td>
             <td>
-                <p>Defines a filter condition for the current pattern state. Only if an event passes the filter, it can match the state:</p>
+                <p>Defines a condition for the current pattern state. Only if an event satisifes the condition, it can match the state:</p>
 {% highlight java %}
-patternState.where(new FilterFunction<Event>() {
+patternState.where(new IterativeCondition<Event>() {
     @Override
-    public boolean filter(Event value) throws Exception {
+    public boolean filter(Event value, Context ctx) throws Exception {
         return ... // some condition
     }
 });
@@ -310,14 +368,14 @@ patternState.where(new FilterFunction<Event>() {
             <td>
                 <p>Adds a new filter condition which is ORed with an existing filter condition. Only if an event passes the filter condition, it can match the state:</p>
 {% highlight java %}
-patternState.where(new FilterFunction<Event>() {
+patternState.where(new IterativeCondition<Event>() {
     @Override
-    public boolean filter(Event value) throws Exception {
+    public boolean filter(Event value, Context ctx) throws Exception {
         return ... // some condition
     }
-}).or(new FilterFunction<Event>() {
+}).or(new IterativeCondition<Event>() {
     @Override
-    public boolean filter(Event value) throws Exception {
+    public boolean filter(Event value, Context ctx) throws Exception {
         return ... // alternative condition
     }
 });
@@ -684,12 +742,12 @@ DataStream<Event> partitionedInput = input.keyBy(new KeySelector<Event, Integer>
 });
 
 Pattern<Event, ?> pattern = Pattern.<Event>begin("start")
-	.next("middle").where(new FilterFunction<Event>() {
+	.next("middle").where(new SimpleCondition<Event>() {
 		@Override
 		public boolean filter(Event value) throws Exception {
 			return value.getName().equals("error");
 		}
-	}).followedBy("end").where(new FilterFunction<Event>() {
+	}).followedBy("end").where(new SimpleCondition<Event>() {
 		@Override
 		public boolean filter(Event value) throws Exception {
 			return value.getName().equals("critical");

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
index 5baf780..a1db460 100644
--- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
+++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala
@@ -17,8 +17,9 @@
  */
 package org.apache.flink.cep.scala.pattern
 
-import org.apache.flink.api.common.functions.FilterFunction
 import org.apache.flink.cep
+import org.apache.flink.cep.pattern.conditions.IterativeCondition
+import org.apache.flink.cep.pattern.conditions.IterativeCondition.Context
 import org.apache.flink.cep.pattern.{Quantifier, Pattern => JPattern}
 import org.apache.flink.streaming.api.windowing.time.Time
 
@@ -67,8 +68,8 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     *
     * @return Filter condition for an event to be matched
     */
-  def getFilterFunction(): Option[FilterFunction[F]] = {
-    Option(jPattern.getFilterFunction())
+  def getCondition(): Option[IterativeCondition[F]] = {
+    Option(jPattern.getCondition())
   }
 
   /**
@@ -127,7 +128,7 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     * @param filter Filter condition
     * @return The same pattern operator where the new filter condition is set
     */
-  def where(filter: FilterFunction[F]): Pattern[T, F] = {
+  def where(filter: IterativeCondition[F]): Pattern[T, F] = {
     jPattern.where(filter)
     this
   }
@@ -138,7 +139,7 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     * @param filter Or filter function
     * @return The same pattern operator where the new filter condition is set
     */
-  def or(filter: FilterFunction[F]): Pattern[T, F] = {
+  def or(filter: IterativeCondition[F]): Pattern[T, F] = {
     jPattern.or(filter)
     this
   }
@@ -149,11 +150,26 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) {
     * @param filterFun Filter condition
     * @return The same pattern operator where the new filter condition is set
     */
+  def where(filterFun: (F, Context[F]) => Boolean): Pattern[T, F] = {
+    val filter = new IterativeCondition[F] {
+      val cleanFilter = cep.scala.cleanClosure(filterFun)
+
+      override def filter(value: F, ctx: Context[F]): Boolean = cleanFilter(value, ctx)
+    }
+    where(filter)
+  }
+
+  /**
+    * Specifies a filter condition which has to be fulfilled by an event in order to be matched.
+    *
+    * @param filterFun Filter condition
+    * @return The same pattern operator where the new filter condition is set
+    */
   def where(filterFun: F => Boolean): Pattern[T, F] = {
-    val filter = new FilterFunction[F] {
+    val filter = new IterativeCondition[F] {
       val cleanFilter = cep.scala.cleanClosure(filterFun)
 
-      override def filter(value: F): Boolean = cleanFilter(value)
+      override def filter(value: F, ctx: Context[F]): Boolean = cleanFilter(value)
     }
     where(filter)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/pattern/PatternTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/pattern/PatternTest.scala b/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/pattern/PatternTest.scala
index 5f49031..a95dddd 100644
--- a/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/pattern/PatternTest.scala
+++ b/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/pattern/PatternTest.scala
@@ -17,12 +17,13 @@
  */
 package org.apache.flink.cep.scala.pattern
 
-import org.apache.flink.api.common.functions.FilterFunction
-import org.apache.flink.cep.pattern.{AndFilterFunction, SubtypeFilterFunction, Pattern => JPattern}
+import org.apache.flink.cep.pattern.{Pattern => JPattern}
 import org.junit.Assert._
 import org.junit.Test
 import org.apache.flink.cep.Event
 import org.apache.flink.cep.SubEvent
+import org.apache.flink.cep.pattern.conditions.IterativeCondition.Context
+import org.apache.flink.cep.pattern.conditions._
 
 class PatternTest {
 
@@ -80,19 +81,19 @@ class PatternTest {
   def testStrictContiguityWithCondition: Unit = {
     val pattern = Pattern.begin[Event]("start")
       .next("next")
-      .where((value: Event) => value.getName() == "foobar")
+      .where((value: Event, ctx: Context[Event]) => value.getName() == "foobar")
       .next("end")
-      .where((value: Event) => value.getId() == 42)
+      .where((value: Event, ctx: Context[Event]) => value.getId() == 42)
 
     val jPattern = JPattern.begin[Event]("start")
       .next("next")
-      .where(new FilterFunction[Event]() {
+      .where(new SimpleCondition[Event]() {
         @throws[Exception]
         def filter(value: Event): Boolean = {
           return value.getName() == "foobar"
         }
       }).next("end")
-      .where(new FilterFunction[Event]() {
+      .where(new SimpleCondition[Event]() {
         @throws[Exception]
         def filter(value: Event): Boolean = {
           return value.getId() == 42
@@ -109,9 +110,9 @@ class PatternTest {
     assertTrue(previous.getPrevious.isDefined)
     assertFalse(preprevious.getPrevious.isDefined)
 
-    assertTrue(pattern.getFilterFunction.isDefined)
-    assertTrue(previous.getFilterFunction.isDefined)
-    assertFalse(preprevious.getFilterFunction.isDefined)
+    assertTrue(pattern.getCondition.isDefined)
+    assertTrue(previous.getCondition.isDefined)
+    assertFalse(preprevious.getCondition.isDefined)
 
     assertEquals(pattern.getName, "end")
     assertEquals(previous.getName, "next")
@@ -140,8 +141,8 @@ class PatternTest {
     assertTrue(previous.getPrevious.isDefined)
     assertFalse(preprevious.getPrevious.isDefined)
 
-    assertTrue(previous.getFilterFunction.isDefined)
-    assertTrue(previous.getFilterFunction.get.isInstanceOf[SubtypeFilterFunction[_]])
+    assertTrue(previous.getCondition.isDefined)
+    assertTrue(previous.getCondition.get.isInstanceOf[SubtypeCondition[_]])
 
     assertEquals(pattern.getName, "end")
     assertEquals(previous.getName, "subevent")
@@ -159,7 +160,7 @@ class PatternTest {
     val jpattern = JPattern.begin[Event]("start")
       .next("subevent")
       .subtype(classOf[SubEvent])
-      .where(new FilterFunction[SubEvent]() {
+      .where(new SimpleCondition[SubEvent]() {
         @throws[Exception]
         def filter(value: SubEvent): Boolean = {
           return false
@@ -178,7 +179,7 @@ class PatternTest {
     assertFalse(preprevious.getPrevious.isDefined)
 
     assertTrue(pattern.isInstanceOf[FollowedByPattern[_, _]])
-    assertTrue(previous.getFilterFunction.isDefined)
+    assertTrue(previous.getCondition().isDefined)
 
     assertEquals(pattern.getName, "end")
     assertEquals(previous.getName, "subevent")
@@ -206,8 +207,8 @@ class PatternTest {
       jPattern.getClass().getSimpleName())
       //best effort to confirm congruent filter functions
       && compareFilterFunctions(
-      pattern.getFilterFunction.orNull,
-      jPattern.getFilterFunction())
+      pattern.getCondition().orNull,
+      jPattern.getCondition())
       //recursively check previous patterns
       && checkCongruentRepresentations(
       pattern.getPrevious.orNull,
@@ -218,7 +219,8 @@ class PatternTest {
     a == b && b == c
   }
 
-  def compareFilterFunctions(sFilter: FilterFunction[_], jFilter: FilterFunction[_]): Boolean = {
+  def compareFilterFunctions(sFilter: IterativeCondition[_],
+                             jFilter: IterativeCondition[_]): Boolean = {
     /**
       * We would like to simply compare the filter functions like this:
       *
@@ -230,16 +232,16 @@ class PatternTest {
       */
     (sFilter, jFilter) match {
       //matching types: and-filter; branch and recurse for inner filters
-      case (saf: AndFilterFunction[_], jaf: AndFilterFunction[_])
+      case (saf: AndCondition[_], jaf: AndCondition[_])
       => (compareFilterFunctions(saf.getLeft(), jaf.getLeft())
         && compareFilterFunctions(saf.getRight(), jaf.getRight()))
       //matching types: subtype-filter
-      case (saf: SubtypeFilterFunction[_], jaf: SubtypeFilterFunction[_]) => true
+      case (saf: SubtypeCondition[_], jaf: SubtypeCondition[_]) => true
       //mismatch: one-sided and/subtype-filter
-      case (_: AndFilterFunction[_] | _: SubtypeFilterFunction[_], _) => false
-      case (_, _: AndFilterFunction[_] | _: SubtypeFilterFunction[_]) => false
+      case (_: AndCondition[_] | _: SubtypeCondition[_], _) => false
+      case (_, _: AndCondition[_] | _: SubtypeCondition[_]) => false
       //from here we can only check mutual presence or absence of a function
-      case (s: FilterFunction[_], j: FilterFunction[_]) => true
+      case (s: IterativeCondition[_], j: IterativeCondition[_]) => true
       case (null, null) => true
       case _ => false
     }

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
index 445d038..80227fc 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
@@ -18,8 +18,14 @@
 
 package org.apache.flink.cep.nfa;
 
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.util.Preconditions;
 
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Helper class which encapsulates the state of the NFA computation. It points to the current state,
  * the last taken event, its occurrence timestamp, the current version and the starting timestamp
@@ -45,7 +51,10 @@ public class ComputationState<T> {
 
 	private final State<T> previousState;
 
+	private final ConditionContext conditionContext;
+
 	private ComputationState(
+			final NFA<T> nfa,
 			final State<T> currentState,
 			final State<T> previousState,
 			final T event,
@@ -58,6 +67,11 @@ public class ComputationState<T> {
 		this.version = version;
 		this.startTimestamp = startTimestamp;
 		this.previousState = previousState;
+		this.conditionContext = new ConditionContext(nfa, this);
+	}
+
+	public ConditionContext getConditionContext() {
+		return conditionContext;
 	}
 
 	public boolean isFinalState() {
@@ -92,23 +106,80 @@ public class ComputationState<T> {
 		return version;
 	}
 
-	public static <T> ComputationState<T> createStartState(final State<T> state) {
+	public static <T> ComputationState<T> createStartState(final NFA<T> nfa, final State<T> state) {
 		Preconditions.checkArgument(state.isStart());
-		return new ComputationState<>(state, null, null, -1L, new DeweyNumber(1), -1L);
+		return new ComputationState<>(nfa, state, null, null, -1L, new DeweyNumber(1), -1L);
 	}
 
-	public static <T> ComputationState<T> createStartState(final State<T> state, final DeweyNumber version) {
+	public static <T> ComputationState<T> createStartState(final NFA<T> nfa, final State<T> state, final DeweyNumber version) {
 		Preconditions.checkArgument(state.isStart());
-		return new ComputationState<>(state, null, null, -1L, version, -1L);
+		return new ComputationState<>(nfa, state, null, null, -1L, version, -1L);
 	}
 
 	public static <T> ComputationState<T> createState(
+			final NFA<T> nfa,
 			final State<T> currentState,
 			final State<T> previousState,
 			final T event,
 			final long timestamp,
 			final DeweyNumber version,
 			final long startTimestamp) {
-		return new ComputationState<>(currentState, previousState, event, timestamp, version, startTimestamp);
+		return new ComputationState<>(nfa, currentState, previousState, event, timestamp, version, startTimestamp);
+	}
+
+	/**
+	 * The context used when evaluating this computation state.
+	 */
+	public class ConditionContext implements IterativeCondition.Context<T> {
+
+		private static final long serialVersionUID = -6733978464782277795L;
+
+		/**
+		 * A flag indicating if we should recompute the matching pattern, so that
+		 * the {@link IterativeCondition iterative condition} can be evaluated.
+		 */
+		private boolean shouldUpdate;
+
+		/** The current computation state. */
+		private transient ComputationState<T> computationState;
+
+		/** The owning {@link NFA} of this computation state. */
+		private final NFA<T> nfa;
+
+		/**
+		 * The matched pattern so far. A condition will be evaluated over this
+		 * pattern. This is evaluated <b>only once</b>, as this is an expensive
+		 * operation that traverses a path in the {@link SharedBuffer}.
+		 */
+		private transient Map<String, List<T>> matchedEvents;
+
+		public ConditionContext(NFA<T> nfa, ComputationState<T> computationState) {
+			this.nfa = nfa;
+			this.computationState = computationState;
+			this.shouldUpdate = true;
+		}
+
+		@Override
+		public Iterable<T> getEventsForPattern(final String key) {
+			Preconditions.checkNotNull(key);
+
+			// the (partially) matched pattern is computed lazily when this method is called.
+			// this is to avoid any overheads when using a simple, non-iterative condition.
+
+			if (shouldUpdate) {
+				this.matchedEvents = nfa.extractCurrentMatches(computationState);
+				shouldUpdate = false;
+			}
+
+			return new Iterable<T>() {
+				@Override
+				public Iterator<T> iterator() {
+					List<T> elements = matchedEvents.get(key);
+					return elements == null
+							? Collections.EMPTY_LIST.<T>iterator()
+							: elements.iterator();
+				}
+			};
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index ab03566..cddc1ed 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -21,18 +21,19 @@ package org.apache.flink.cep.nfa;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.LinkedHashMultimap;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream;
 import org.apache.flink.cep.NonDuplicatingTypeSerializer;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.util.Preconditions;
 
 import javax.annotation.Nullable;
 import java.io.ByteArrayInputStream;
@@ -140,10 +141,9 @@ public class NFA<T> implements Serializable {
 		this.nonDuplicatingTypeSerializer = new NonDuplicatingTypeSerializer<>(eventSerializer);
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
-		stringSharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
-		computationStates = new LinkedList<>();
-
-		states = new HashSet<>();
+		this.stringSharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
+		this.computationStates = new LinkedList<>();
+		this.states = new HashSet<>();
 	}
 
 	public Set<State<T>> getStates() {
@@ -160,7 +160,7 @@ public class NFA<T> implements Serializable {
 		states.add(state);
 
 		if (state.isStart()) {
-			computationStates.add(ComputationState.createStartState(state));
+			computationStates.add(ComputationState.createStartState(this, state));
 		}
 	}
 
@@ -214,10 +214,6 @@ public class NFA<T> implements Serializable {
 						computationState.getPreviousState().getName(),
 						computationState.getEvent(),
 						computationState.getTimestamp());
-				stringSharedBuffer.remove(
-						computationState.getPreviousState().getName(),
-						computationState.getEvent(),
-						computationState.getTimestamp());
 
 				newComputationStates = Collections.emptyList();
 			} else if (event != null) {
@@ -233,8 +229,10 @@ public class NFA<T> implements Serializable {
 					result.addAll(matches);
 
 					// remove found patterns because they are no longer needed
-					stringSharedBuffer.release(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
-					stringSharedBuffer.remove(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
+					stringSharedBuffer.release(
+							newComputationState.getPreviousState().getName(),
+							newComputationState.getEvent(),
+							newComputationState.getTimestamp());
 				} else {
 					// add new computation state; it will be processed once the next event arrives
 					computationStates.add(newComputationState);
@@ -332,23 +330,29 @@ public class NFA<T> implements Serializable {
 	/**
 	 * Computes the next computation states based on the given computation state, the current event,
 	 * its timestamp and the internal state machine. The algorithm is:
-	 *
-	 * 1. Decide on valid transitions and number of branching paths. See {@link OutgoingEdges}
-	 * 2. Perform transitions:
-	 *      a) IGNORE (links in {@link SharedBuffer} will still point to the previous event)
-	 *          - do not perform for Start State - special case
-	 *          - if stays in the same state increase the current stage for future use with number of
-	 *            outgoing edges
-	 *          - if after PROCEED increase current stage and add new stage (as we change the state)
-	 *          - lock the entry in {@link SharedBuffer} as it is needed in the created branch
-	 *      b) TAKE (links in {@link SharedBuffer} will point to the current event)
-	 *          - add entry to the shared buffer with version of the current computation state
-	 *          - add stage and then increase with number of takes for the future computation states
-	 *          - peek to the next state if it has PROCEED path to a Final State, if true create
-	 *            Final ComputationState to emit results
-	 * 3. Handle the Start State, as it always have to remain
-	 * 4. Release the corresponding entries in {@link SharedBuffer}.
-	 *
+	 *<ol>
+	 *     <li>Decide on valid transitions and number of branching paths. See {@link OutgoingEdges}</li>
+	 * 	   <li>Perform transitions:
+	 * 	   	<ol>
+	 *          <li>IGNORE (links in {@link SharedBuffer} will still point to the previous event)</li>
+	 *          <ul>
+	 *              <li>do not perform for Start State - special case</li>
+	 *          	<li>if stays in the same state increase the current stage for future use with number of outgoing edges</li>
+	 *          	<li>if after PROCEED increase current stage and add new stage (as we change the state)</li>
+	 *          	<li>lock the entry in {@link SharedBuffer} as it is needed in the created branch</li>
+	 *      	</ul>
+	 *      	<li>TAKE (links in {@link SharedBuffer} will point to the current event)</li>
+	 *          <ul>
+	 *              <li>add entry to the shared buffer with version of the current computation state</li>
+	 *              <li>add stage and then increase with number of takes for the future computation states</li>
+	 *              <li>peek to the next state if it has PROCEED path to a Final State, if true create Final
+	 *              ComputationState to emit results</li>
+	 *          </ul>
+	 *      </ol>
+	 *     </li>
+	 * 	   <li>Handle the Start State, as it always have to remain </li>
+	 *     <li>Release the corresponding entries in {@link SharedBuffer}.</li>
+	 *</ol>
 	 *
 	 * @param computationState Current computation state
 	 * @param event Current event which is processed
@@ -387,85 +391,81 @@ public class NFA<T> implements Serializable {
 							ignoreBranchesToVisit--;
 						}
 
-						resultingComputationStates.add(
-							ComputationState.createState(
+						addComputationState(
+								resultingComputationStates,
 								edge.getTargetState(),
 								computationState.getPreviousState(),
 								computationState.getEvent(),
 								computationState.getTimestamp(),
 								version,
 								computationState.getStartTimestamp()
-							)
 						);
-						stringSharedBuffer.lock(
-							computationState.getPreviousState().getName(),
-							computationState.getEvent(),
-							computationState.getTimestamp());
 					}
 				}
 				break;
 				case TAKE:
-					final State<T> newState = edge.getTargetState();
-					final State<T> consumingState = edge.getSourceState();
-					final State<T> previousEventState = computationState.getPreviousState();
+					final State<T> nextState = edge.getTargetState();
+					final State<T> currentState = edge.getSourceState();
+					final State<T> previousState = computationState.getPreviousState();
 
 					final T previousEvent = computationState.getEvent();
-					final DeweyNumber currentVersion = computationState.getVersion();
 
-					final DeweyNumber newComputationStateVersion = new DeweyNumber(currentVersion).addStage().increase(takeBranchesToVisit);
+					final DeweyNumber currentVersion = computationState.getVersion();
+					final DeweyNumber nextVersion = new DeweyNumber(currentVersion).addStage().increase(takeBranchesToVisit);
 					takeBranchesToVisit--;
 
 					final long startTimestamp;
 					if (computationState.isStartState()) {
 						startTimestamp = timestamp;
 						stringSharedBuffer.put(
-							consumingState.getName(),
+							currentState.getName(),
 							event,
 							timestamp,
 							currentVersion);
 					} else {
 						startTimestamp = computationState.getStartTimestamp();
 						stringSharedBuffer.put(
-							consumingState.getName(),
+							currentState.getName(),
 							event,
 							timestamp,
-							previousEventState.getName(),
+							previousState.getName(),
 							previousEvent,
 							computationState.getTimestamp(),
 							currentVersion);
 					}
 
-					// a new computation state is referring to the shared entry
-					stringSharedBuffer.lock(consumingState.getName(), event, timestamp);
-
-					resultingComputationStates.add(ComputationState.createState(
-						newState,
-						consumingState,
-						event,
-						timestamp,
-						newComputationStateVersion,
-						startTimestamp
-					));
+					addComputationState(
+							resultingComputationStates,
+							nextState,
+							currentState,
+							event,
+							timestamp,
+							nextVersion,
+							startTimestamp);
 
 					//check if newly created state is optional (have a PROCEED path to Final state)
-					final State<T> finalState = findFinalStateAfterProceed(newState, event);
+					final State<T> finalState = findFinalStateAfterProceed(nextState, event, computationState);
 					if (finalState != null) {
-						stringSharedBuffer.lock(consumingState.getName(), event, timestamp);
-						resultingComputationStates.add(ComputationState.createState(
-							finalState,
-							consumingState,
-							event,
-							timestamp,
-							newComputationStateVersion,
-							startTimestamp));
+						addComputationState(
+								resultingComputationStates,
+								finalState,
+								currentState,
+								event,
+								timestamp,
+								nextVersion,
+								startTimestamp);
 					}
 					break;
 			}
 		}
 
 		if (computationState.isStartState()) {
-			final int totalBranches = calculateIncreasingSelfState(outgoingEdges.getTotalIgnoreBranches(), outgoingEdges.getTotalTakeBranches());
-			final ComputationState<T> startState = createStartState(computationState, totalBranches);
+			int totalBranches = calculateIncreasingSelfState(
+					outgoingEdges.getTotalIgnoreBranches(),
+					outgoingEdges.getTotalTakeBranches());
+
+			DeweyNumber startVersion = computationState.getVersion().increase(totalBranches);
+			ComputationState<T> startState = ComputationState.createStartState(this, computationState.getState(), startVersion);
 			resultingComputationStates.add(startState);
 		}
 
@@ -475,17 +475,26 @@ public class NFA<T> implements Serializable {
 				computationState.getPreviousState().getName(),
 				computationState.getEvent(),
 				computationState.getTimestamp());
-			// try to remove unnecessary shared buffer entries
-			stringSharedBuffer.remove(
-				computationState.getPreviousState().getName(),
-				computationState.getEvent(),
-				computationState.getTimestamp());
 		}
 
 		return resultingComputationStates;
 	}
 
-	private State<T> findFinalStateAfterProceed(State<T> state, T event) {
+	private void addComputationState(
+			List<ComputationState<T>> computationStates,
+			State<T> currentState,
+			State<T> previousState,
+			T event,
+			long timestamp,
+			DeweyNumber version,
+			long startTimestamp) {
+		ComputationState<T> computationState = ComputationState.createState(
+				this, currentState, previousState, event, timestamp, version, startTimestamp);
+		computationStates.add(computationState);
+		stringSharedBuffer.lock(previousState.getName(), event, timestamp);
+	}
+
+	private State<T> findFinalStateAfterProceed(State<T> state, T event, ComputationState<T> computationState) {
 		final Stack<State<T>> statesToCheck = new Stack<>();
 		statesToCheck.push(state);
 
@@ -494,7 +503,7 @@ public class NFA<T> implements Serializable {
 				final State<T> currentState = statesToCheck.pop();
 				for (StateTransition<T> transition : currentState.getStateTransitions()) {
 					if (transition.getAction() == StateTransitionAction.PROCEED &&
-						checkFilterCondition(transition.getCondition(), event)) {
+							checkFilterCondition(computationState, transition.getCondition(), event)) {
 						if (transition.getTargetState().isFinal()) {
 							return transition.getTargetState();
 						} else {
@@ -514,15 +523,12 @@ public class NFA<T> implements Serializable {
 		return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + 1;
 	}
 
-	private ComputationState<T> createStartState(final ComputationState<T> computationState, final int totalBranches) {
-		final DeweyNumber startVersion = computationState.getVersion().increase(totalBranches);
-		return ComputationState.createStartState(computationState.getState(), startVersion);
-	}
-
 	private OutgoingEdges<T> createDecisionGraph(ComputationState<T> computationState, T event) {
+		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(computationState.getState());
+
 		final Stack<State<T>> states = new Stack<>();
 		states.push(computationState.getState());
-		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(computationState.getState());
+
 		//First create all outgoing edges, so to be able to reason about the Dewey version
 		while (!states.isEmpty()) {
 			State<T> currentState = states.pop();
@@ -531,7 +537,7 @@ public class NFA<T> implements Serializable {
 			// check all state transitions for each state
 			for (StateTransition<T> stateTransition : stateTransitions) {
 				try {
-					if (checkFilterCondition(stateTransition.getCondition(), event)) {
+					if (checkFilterCondition(computationState, stateTransition.getCondition(), event)) {
 						// filter condition is true
 						switch (stateTransition.getAction()) {
 							case PROCEED:
@@ -553,9 +559,38 @@ public class NFA<T> implements Serializable {
 		return outgoingEdges;
 	}
 
+	private boolean checkFilterCondition(ComputationState<T> computationState, IterativeCondition<T> condition, T event) throws Exception {
+		return condition == null || condition.filter(event, computationState.getConditionContext());
+	}
+
+	Map<String, List<T>> extractCurrentMatches(final ComputationState<T> computationState) {
+		if (computationState.getPreviousState() == null) {
+			return new HashMap<>();
+		}
+
+		Collection<LinkedHashMultimap<String, T>> paths = stringSharedBuffer.extractPatterns(
+				computationState.getPreviousState().getName(),
+				computationState.getEvent(),
+				computationState.getTimestamp(),
+				computationState.getVersion());
 
-	private boolean checkFilterCondition(FilterFunction<T> condition, T event) throws Exception {
-		return condition == null || condition.filter(event);
+		// for a given computation state, we cannot have more than one matching patterns.
+		Preconditions.checkArgument(paths.size() <= 1);
+
+		TypeSerializer<T> serializer = nonDuplicatingTypeSerializer.getTypeSerializer();
+
+		Map<String, List<T>> result = new HashMap<>();
+		for (LinkedHashMultimap<String, T> path: paths) {
+			for (String key: path.keySet()) {
+				Set<T> events = path.get(key);
+				List<T> values = new ArrayList<>(events.size());
+				for (T event: events) {
+					values.add(serializer.isImmutableType() ? event : serializer.copy(event));
+				}
+				result.put(key, values);
+			}
+		}
+		return result;
 	}
 
 	/**
@@ -573,6 +608,9 @@ public class NFA<T> implements Serializable {
 			computationState.getTimestamp(),
 			computationState.getVersion());
 
+		// for a given computation state, we cannot have more than one matching patterns.
+		Preconditions.checkArgument(paths.size() <= 1);
+
 		List<Map<String, T>> result = new ArrayList<>();
 
 		TypeSerializer<T> serializer = nonDuplicatingTypeSerializer.getTypeSerializer();
@@ -601,6 +639,28 @@ public class NFA<T> implements Serializable {
 		return result;
 	}
 
+	/**
+	 * Generates a state name from a given name template and an index.
+	 * <p>
+	 * If the template ends with "[]" the index is inserted in between the square brackets.
+	 * Otherwise, an underscore and the index is appended to the name.
+	 *
+	 * @param name Name template
+	 * @param index Index of the state
+	 * @return Generated state name from the given state name template
+	 */
+	static String generateStateName(final String name, final int index) {
+		Matcher matcher = namePattern.matcher(name);
+
+		if (matcher.matches()) {
+			return matcher.group(1) + index + matcher.group(2);
+		} else {
+			return name + "_" + index;
+		}
+	}
+
+	//////////////////////			Fault-Tolerance / Migration			//////////////////////
+
 	private void writeObject(ObjectOutputStream oos) throws IOException {
 		oos.defaultWriteObject();
 
@@ -692,6 +752,7 @@ public class NFA<T> implements Serializable {
 				final State<T> previousState = convertedStates.get(previousName);
 
 				computationStates.add(ComputationState.createState(
+					this,
 					convertedStates.get(currentName),
 					previousState,
 					readState.getEvent(),
@@ -710,6 +771,7 @@ public class NFA<T> implements Serializable {
 		}).getName();
 
 		computationStates.add(ComputationState.createStartState(
+			this,
 			convertedStates.get(startName),
 			new DeweyNumber(this.startEventCounter)));
 
@@ -761,32 +823,13 @@ public class NFA<T> implements Serializable {
 			event = null;
 		}
 
-		return ComputationState.createState(state, previousState, event, timestamp, version, startTimestamp);
+		return ComputationState.createState(this, state, previousState, event, timestamp, version, startTimestamp);
 	}
 
+	//////////////////////			Serialization			//////////////////////
 
 	/**
-	 * Generates a state name from a given name template and an index.
-	 * <p>
-	 * If the template ends with "[]" the index is inserted in between the square brackets.
-	 * Otherwise, an underscore and the index is appended to the name.
-	 *
-	 * @param name Name template
-	 * @param index Index of the state
-	 * @return Generated state name from the given state name template
-	 */
-	static String generateStateName(final String name, final int index) {
-		Matcher matcher = namePattern.matcher(name);
-
-		if (matcher.matches()) {
-			return matcher.group(1) + index + matcher.group(2);
-		} else {
-			return name + "_" + index;
-		}
-	}
-
-	/**
-	 * {@link TypeSerializer} for {@link NFA} that uses Java Serialization.
+	 * A {@link TypeSerializer} for {@link NFA} that uses Java Serialization.
 	 */
 	public static class Serializer<T> extends TypeSerializer<NFA<T>> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index d5b7876..ccc6884 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -52,12 +52,14 @@ import java.util.Stack;
  *
  * The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams".
  *
- * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
+ * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
+ *     https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
  *
  * @param <K> Type of the keys
  * @param <V> Type of the values
  */
 public class SharedBuffer<K extends Serializable, V> implements Serializable {
+
 	private static final long serialVersionUID = 9213251042562206495L;
 
 	private final TypeSerializer<V> valueSerializer;
@@ -66,20 +68,20 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 
 	public SharedBuffer(final TypeSerializer<V> valueSerializer) {
 		this.valueSerializer = valueSerializer;
-		pages = new HashMap<>();
+		this.pages = new HashMap<>();
 	}
 
 	/**
 	 * Stores given value (value + timestamp) under the given key. It assigns a preceding element
 	 * relation to the entry which is defined by the previous key, value (value + timestamp).
 	 *
-	 * @param key Key of the current value
-	 * @param value Current value
-	 * @param timestamp Timestamp of the current value (a value requires always a timestamp to make it uniquely referable))
-	 * @param previousKey Key of the value for the previous relation
-	 * @param previousValue Value for the previous relation
+	 * @param key               Key of the current value
+	 * @param value             Current value
+	 * @param timestamp         Timestamp of the current value (a value requires always a timestamp to make it uniquely referable))
+	 * @param previousKey       Key of the value for the previous relation
+	 * @param previousValue     Value for the previous relation
 	 * @param previousTimestamp Timestamp of the value for the previous relation
-	 * @param version Version of the previous relation
+	 * @param version           Version of the previous relation
 	 */
 	public void put(
 			final K key,
@@ -89,14 +91,6 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 			final V previousValue,
 			final long previousTimestamp,
 			final DeweyNumber version) {
-		SharedBufferPage<K, V> page;
-
-		if (!pages.containsKey(key)) {
-			page = new SharedBufferPage<K, V>(key);
-			pages.put(key, page);
-		} else {
-			page = pages.get(key);
-		}
 
 		final SharedBufferEntry<K, V> previousSharedBufferEntry = get(previousKey, previousValue, previousTimestamp);
 
@@ -108,55 +102,41 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 				"relation has been already pruned, even though you expect it to be still there.");
 		}
 
-		page.add(
-			new ValueTimeWrapper<>(value, timestamp),
-			previousSharedBufferEntry,
-			version);
+		put(key, value, timestamp, previousSharedBufferEntry, version);
 	}
 
 	/**
 	 * Stores given value (value + timestamp) under the given key. It assigns no preceding element
 	 * relation to the entry.
 	 *
-	 * @param key Key of the current value
-	 * @param value Current value
+	 * @param key       Key of the current value
+	 * @param value     Current value
 	 * @param timestamp Timestamp of the current value (a value requires always a timestamp to make it uniquely referable))
-	 * @param version Version of the previous relation
+	 * @param version   Version of the previous relation
 	 */
 	public void put(
-		final K key,
-		final V value,
-		final long timestamp,
-		final DeweyNumber version) {
-		SharedBufferPage<K, V> page;
-
-		if (!pages.containsKey(key)) {
-			page = new SharedBufferPage<K, V>(key);
-			pages.put(key, page);
-		} else {
-			page = pages.get(key);
-		}
+			final K key,
+			final V value,
+			final long timestamp,
+			final DeweyNumber version) {
 
-		page.add(
-			new ValueTimeWrapper<>(value, timestamp),
-			null,
-			version);
+		put(key, value, timestamp, null, version);
 	}
 
-	/**
-	 * Checks whether the given key, value, timestamp triple is contained in the shared buffer
-	 *
-	 * @param key Key of the value
-	 * @param value Value
-	 * @param timestamp Timestamp of the value
-	 * @return Whether a value with the given timestamp is registered under the given key
-	 */
-	public boolean contains(
-		final K key,
-		final V value,
-		final long timestamp) {
+	private void put(
+			final K key,
+			final V value,
+			final long timestamp,
+			final SharedBufferEntry<K, V> previousSharedBufferEntry,
+			final DeweyNumber version) {
 
-		return pages.containsKey(key) && pages.get(key).contains(new ValueTimeWrapper<>(value, timestamp));
+		SharedBufferPage<K, V> page = pages.get(key);
+		if (page == null) {
+			page = new SharedBufferPage<>(key);
+			pages.put(key, page);
+		}
+
+		page.add(new ValueTimeWrapper<>(value, timestamp), previousSharedBufferEntry, version);
 	}
 
 	public boolean isEmpty() {
@@ -272,47 +252,29 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 	 * Increases the reference counter for the given value, key, timestamp entry so that it is not
 	 * accidentally removed.
 	 *
-	 * @param key Key of the value to lock
-	 * @param value Value to lock
+	 * @param key       Key of the value to lock
+	 * @param value     Value to lock
 	 * @param timestamp Timestamp of the value to lock
 	 */
 	public void lock(final K key, final V value, final long timestamp) {
 		SharedBufferEntry<K, V> entry = get(key, value, timestamp);
-
 		if (entry != null) {
 			entry.increaseReferenceCounter();
 		}
 	}
 
 	/**
-	 * Decreases the reference counter for the given value, key, timstamp entry so that it can be
+	 * Decreases the reference counter for the given value, key, timestamp entry so that it can be
 	 * removed once the reference counter reaches 0.
 	 *
-	 * @param key Key of the value to release
-	 * @param value Value to release
+	 * @param key       Key of the value to release
+	 * @param value     Value to release
 	 * @param timestamp Timestamp of the value to release
 	 */
 	public void release(final K key, final V value, final long timestamp) {
 		SharedBufferEntry<K, V> entry = get(key, value, timestamp);
-
-		if (entry != null ) {
-			entry.decreaseReferenceCounter();
-		}
-	}
-
-	/**
-	 * Removes the given value, key, timestamp entry if its reference counter is 0. It will also
-	 * release the next element in its previous relation and apply remove to this element
-	 * recursively.
-	 *
-	 * @param key Key of the value to remove
-	 * @param value Value to remove
-	 * @param timestamp Timestamp of the value to remvoe
-	 */
-	public void remove(final K key, final V value, final long timestamp) {
-		SharedBufferEntry<K, V> entry = get(key, value, timestamp);
-
 		if (entry != null) {
+			entry.decreaseReferenceCounter();
 			internalRemove(entry);
 		}
 	}
@@ -626,10 +588,6 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 			sharedBufferEntry.addEdge(newEdge);
 		}
 
-		public boolean contains(final ValueTimeWrapper<V> valueTime) {
-			return entries.containsKey(valueTime);
-		}
-
 		public SharedBufferEntry<K, V> get(final ValueTimeWrapper<V> valueTime) {
 			return entries.get(valueTime);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
index 27e0dcd..c673576 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;
@@ -66,29 +66,29 @@ public class State<T> implements Serializable {
 	}
 
 	private void addStateTransition(
-		final StateTransitionAction action,
-		final State<T> targetState,
-		final FilterFunction<T> condition) {
+			final StateTransitionAction action,
+			final State<T> targetState,
+			final IterativeCondition<T> condition) {
 		stateTransitions.add(new StateTransition<T>(this, action, targetState, condition));
 	}
 
-	public void addIgnore(final FilterFunction<T> condition) {
+	public void addIgnore(final IterativeCondition<T> condition) {
 		addStateTransition(StateTransitionAction.IGNORE, this, condition);
 	}
 
-	public void addIgnore(final State<T> targetState,final FilterFunction<T> condition) {
+	public void addIgnore(final State<T> targetState,final IterativeCondition<T> condition) {
 		addStateTransition(StateTransitionAction.IGNORE, targetState, condition);
 	}
 
-	public void addTake(final State<T> targetState, final FilterFunction<T> condition) {
+	public void addTake(final State<T> targetState, final IterativeCondition<T> condition) {
 		addStateTransition(StateTransitionAction.TAKE, targetState, condition);
 	}
 
-	public void addProceed(final State<T> targetState, final FilterFunction<T> condition) {
+	public void addProceed(final State<T> targetState, final IterativeCondition<T> condition) {
 		addStateTransition(StateTransitionAction.PROCEED, targetState, condition);
 	}
 
-	public void addTake(final FilterFunction<T> condition) {
+	public void addTake(final IterativeCondition<T> condition) {
 		addStateTransition(StateTransitionAction.TAKE, this, condition);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
index e3c7b7a..f80edfc 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/StateTransition.java
@@ -19,6 +19,8 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 
 import java.io.Serializable;
 import java.util.Objects;
@@ -29,17 +31,24 @@ public class StateTransition<T> implements Serializable {
 	private final StateTransitionAction action;
 	private final State<T> sourceState;
 	private final State<T> targetState;
-	private final FilterFunction<T> condition;
+	private IterativeCondition<T> newCondition;
+
+	/**
+	 * @deprecated 	This field remains for backwards compatibility.
+	 * Now the conditions extend the {@link IterativeCondition}.
+	 */
+	@Deprecated
+	private FilterFunction<T> condition;
 
 	public StateTransition(
-		final State<T> sourceState,
-		final StateTransitionAction action,
-		final State<T> targetState,
-		final FilterFunction<T> condition) {
+			final State<T> sourceState,
+			final StateTransitionAction action,
+			final State<T> targetState,
+			final IterativeCondition<T> condition) {
 		this.action = action;
 		this.targetState = targetState;
 		this.sourceState = sourceState;
-		this.condition = condition;
+		this.newCondition = condition;
 	}
 
 	public StateTransitionAction getAction() {
@@ -54,8 +63,12 @@ public class StateTransition<T> implements Serializable {
 		return sourceState;
 	}
 
-	public FilterFunction<T> getCondition() {
-		return condition;
+	public IterativeCondition<T> getCondition() {
+		if (condition != null) {
+			this.newCondition = new FilterWrapper<>(condition);
+			this.condition = null;
+		}
+		return newCondition;
 	}
 
 	@Override
@@ -87,7 +100,7 @@ public class StateTransition<T> implements Serializable {
 			.append(sourceState.getName()).append(", ")
 			.append(targetState.getName());
 
-		if (condition != null) {
+		if (newCondition != null) {
 			builder.append(", with filter)");
 		} else {
 			builder.append(")");
@@ -95,4 +108,24 @@ public class StateTransition<T> implements Serializable {
 
 		return builder.toString();
 	}
+
+	/**
+	 * A wrapper to transform a {@link FilterFunction} into a {@link SimpleCondition}.
+	 * This is used only when migrating from an older Flink version.
+	 */
+	private static class FilterWrapper<T> extends SimpleCondition<T> {
+
+		private static final long serialVersionUID = -4973016745698940430L;
+
+		private final FilterFunction<T> filterFunction;
+
+		FilterWrapper(FilterFunction<T> filterFunction) {
+			this.filterFunction = filterFunction;
+		}
+
+		@Override
+		public boolean filter(T value) throws Exception {
+			return filterFunction.filter(value);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 8bd8612..4fb918f 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -21,19 +21,19 @@ package org.apache.flink.cep.nfa.compiler;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.State;
 import org.apache.flink.cep.nfa.StateTransition;
 import org.apache.flink.cep.nfa.StateTransitionAction;
-import org.apache.flink.cep.pattern.FilterFunctions;
+import org.apache.flink.cep.pattern.conditions.BooleanConditions;
 import org.apache.flink.cep.pattern.FollowedByPattern;
 import org.apache.flink.cep.pattern.MalformedPatternException;
-import org.apache.flink.cep.pattern.NotFilterFunction;
+import org.apache.flink.cep.pattern.conditions.NotCondition;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.Quantifier;
 import org.apache.flink.cep.pattern.Quantifier.QuantifierProperty;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
 import javax.annotation.Nullable;
@@ -240,7 +240,7 @@ public class NFACompiler {
 
 		/**
 		 * Converts the given state into a "complex" state consisting of given number of states with
-		 * same {@link FilterFunction}
+		 * same {@link IterativeCondition}
 		 *
 		 * @param sourceState the state to be converted
 		 * @param sinkState the state that the converted state should point to
@@ -271,8 +271,9 @@ public class NFACompiler {
 		@SuppressWarnings("unchecked")
 		private void convertToSingletonState(final State<T> sourceState, final State<T> sinkState) {
 
-			final FilterFunction<T> currentFilterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
-			final FilterFunction<T> trueFunction = FilterFunctions.trueFunction();
+			final IterativeCondition<T> currentFilterFunction = (IterativeCondition<T>) currentPattern.getCondition();
+			final IterativeCondition<T> trueFunction = BooleanConditions.trueFunction();
+
 			sourceState.addTake(sinkState, currentFilterFunction);
 
 			if (currentPattern.getQuantifier() == Quantifier.OPTIONAL) {
@@ -303,16 +304,13 @@ public class NFACompiler {
 		@SuppressWarnings("unchecked")
 		private State<T> createFirstMandatoryStateOfLoop(final State<T> sinkState, final State.StateType stateType) {
 
-			final FilterFunction<T> currentFilterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
+			final IterativeCondition<T> currentFilterFunction = (IterativeCondition<T>) currentPattern.getCondition();
 			final State<T> firstState = new State<>(currentPattern.getName(), stateType);
 
 			firstState.addTake(sinkState, currentFilterFunction);
 			if (currentPattern instanceof FollowedByPattern) {
-				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.EAGER)) {
-					firstState.addIgnore(new NotFilterFunction<>(currentFilterFunction));
-				} else {
-					firstState.addIgnore(FilterFunctions.<T>trueFunction());
-				}
+				final IterativeCondition<T> ignoreCondition = getIgnoreCondition(currentPattern);
+				firstState.addIgnore(ignoreCondition);
 			}
 			return firstState;
 		}
@@ -332,8 +330,8 @@ public class NFACompiler {
 		@SuppressWarnings("unchecked")
 		private void convertToLooping(final State<T> sourceState, final State<T> sinkState, boolean isFirstState) {
 
-			final FilterFunction<T> filterFunction = (FilterFunction<T>) currentPattern.getFilterFunction();
-			final FilterFunction<T> trueFunction = FilterFunctions.<T>trueFunction();
+			final IterativeCondition<T> filterFunction = (IterativeCondition<T>) currentPattern.getCondition();
+			final IterativeCondition<T> trueFunction = BooleanConditions.<T>trueFunction();
 
 			sourceState.addProceed(sinkState, trueFunction);
 			sourceState.addTake(filterFunction);
@@ -342,13 +340,7 @@ public class NFACompiler {
 					currentPattern.getName(),
 					State.StateType.Normal);
 
-
-				final FilterFunction<T> ignoreCondition;
-				if (currentPattern.getQuantifier().hasProperty(QuantifierProperty.EAGER)) {
-					ignoreCondition = new NotFilterFunction<>(filterFunction);
-				} else {
-					ignoreCondition = trueFunction;
-				}
+				final IterativeCondition<T> ignoreCondition = getIgnoreCondition(currentPattern);
 
 				sourceState.addIgnore(ignoreState, ignoreCondition);
 				ignoreState.addTake(sourceState, filterFunction);
@@ -368,6 +360,19 @@ public class NFACompiler {
 		private void convertToLooping(final State<T> sourceState, final State<T> sinkState) {
 			convertToLooping(sourceState, sinkState, false);
 		}
+
+		/**
+		 * @return The {@link IterativeCondition condition} for the {@code IGNORE} edge
+		 * that corresponds to the specified {@link Pattern}. If the pattern is
+		 * {@link QuantifierProperty#EAGER}, the negated user-specified condition is
+		 * returned. In other case, a condition that always evaluated to {@code true} is
+		 * returned.
+		 */
+		private IterativeCondition<T> getIgnoreCondition(Pattern<T, ?> pattern) {
+			return pattern.getQuantifier().hasProperty(QuantifierProperty.EAGER)
+					? new NotCondition<>((IterativeCondition<T>) pattern.getCondition())
+					: BooleanConditions.<T>trueFunction();
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java
index ecaee07..a7391d5 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java
@@ -21,11 +21,15 @@ package org.apache.flink.cep.pattern;
 import org.apache.flink.api.common.functions.FilterFunction;
 
 /**
- * A filter function which combines two filter functions with a logical and. Thus, the filter
+ * @deprecated This is only used when migrating from an older Flink version.
+ * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
+ *
+ * <p>A filter function which combines two filter functions with a logical and. Thus, the filter
  * function only returns true, iff both filters return true.
  *
  * @param <T> Type of the element to filter
  */
+@Deprecated
 public class AndFilterFunction<T> implements FilterFunction<T> {
 	private static final long serialVersionUID = -2109562093871155005L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
deleted file mode 100644
index 12e58ba..0000000
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/FilterFunctions.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.cep.pattern;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-
-public class FilterFunctions<T> {
-
-	private FilterFunctions() {
-	}
-
-	public static <T> FilterFunction<T> trueFunction()  {
-		return new FilterFunction<T>() {
-			@Override
-			public boolean filter(T value) throws Exception {
-				return true;
-			}
-		};
-	}
-
-	public static <T> FilterFunction<T> falseFunction()  {
-		return new FilterFunction<T>() {
-			@Override
-			public boolean filter(T value) throws Exception {
-				return false;
-			}
-		};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
deleted file mode 100644
index a4fc8f5..0000000
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/NotFilterFunction.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cep.pattern;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-
-/**
- * A filter function which negates filter function.
- *
- * @param <T> Type of the element to filter
- */
-public class NotFilterFunction<T> implements FilterFunction<T> {
-	private static final long serialVersionUID = -2109562093871155005L;
-
-	private final FilterFunction<T> original;
-
-	public NotFilterFunction(final FilterFunction<T> original) {
-		this.original = original;
-	}
-
-	@Override
-	public boolean filter(T value) throws Exception {
-		return !original.filter(value);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java
index c42ecb1..3620cae 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/OrFilterFunction.java
@@ -21,11 +21,15 @@ package org.apache.flink.cep.pattern;
 import org.apache.flink.api.common.functions.FilterFunction;
 
 /**
- * A filter function which combines two filter functions with a logical or. Thus, the filter
+ * @deprecated This is only used when migrating from an older Flink version.
+ * Use the {@link org.apache.flink.cep.pattern.conditions.OrCondition} instead.
+ *
+ * <p>A filter function which combines two filter functions with a logical or. Thus, the filter
  * function only returns true, iff at least one of the filter functions holds true.
  *
  * @param <T> Type of the element to filter
  */
+@Deprecated
 public class OrFilterFunction<T> implements FilterFunction<T> {
 	private static final long serialVersionUID = -2109562093871155005L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
index 7b4d9c7..cd51788 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java
@@ -18,9 +18,12 @@
 
 package org.apache.flink.cep.pattern;
 
-import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.java.ClosureCleaner;
 import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.pattern.conditions.AndCondition;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.cep.pattern.conditions.OrCondition;
+import org.apache.flink.cep.pattern.conditions.SubtypeCondition;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.util.Preconditions;
 
@@ -49,7 +52,7 @@ public class Pattern<T, F extends T> {
 	private final Pattern<T, ? extends T> previous;
 
 	// filter condition for an event to be matched
-	private FilterFunction<F> filterFunction;
+	private IterativeCondition<F> condition;
 
 	// window length in which the pattern match has to occur
 	private Time windowTime;
@@ -71,8 +74,8 @@ public class Pattern<T, F extends T> {
 		return previous;
 	}
 
-	public FilterFunction<F> getFilterFunction() {
-		return filterFunction;
+	public IterativeCondition<F> getCondition() {
+		return condition;
 	}
 
 	public Time getWindowTime() {
@@ -90,36 +93,34 @@ public class Pattern<T, F extends T> {
 	/**
 	 * Specifies a filter condition which has to be fulfilled by an event in order to be matched.
 	 *
-	 * @param newFilterFunction Filter condition
+	 * @param condition Filter condition
 	 * @return The same pattern operator where the new filter condition is set
 	 */
-	public Pattern<T, F> where(FilterFunction<F> newFilterFunction) {
-		ClosureCleaner.clean(newFilterFunction, true);
+	public Pattern<T, F> where(IterativeCondition<F> condition) {
+		ClosureCleaner.clean(condition, true);
 
-		if (this.filterFunction == null) {
-			this.filterFunction = newFilterFunction;
+		if (this.condition == null) {
+			this.condition = condition;
 		} else {
-			this.filterFunction = new AndFilterFunction<F>(this.filterFunction, newFilterFunction);
+			this.condition = new AndCondition<>(this.condition, condition);
 		}
-
 		return this;
 	}
 
 	/**
 	 * Specifies a filter condition which is OR'ed with an existing filter function.
 	 *
-	 * @param orFilterFunction OR filter condition
+	 * @param condition OR filter condition
 	 * @return The same pattern operator where the new filter condition is set
 	 */
-	public Pattern<T, F> or(FilterFunction<F> orFilterFunction) {
-		ClosureCleaner.clean(orFilterFunction, true);
+	public Pattern<T, F> or(IterativeCondition<F> condition) {
+		ClosureCleaner.clean(condition, true);
 
-		if (this.filterFunction == null) {
-			this.filterFunction = orFilterFunction;
+		if (this.condition == null) {
+			this.condition = condition;
 		} else {
-			this.filterFunction = new OrFilterFunction<>(this.filterFunction, orFilterFunction);
+			this.condition = new OrCondition<>(this.condition, condition);
 		}
-
 		return this;
 	}
 
@@ -132,10 +133,11 @@ public class Pattern<T, F extends T> {
 	 * @return The same pattern operator with the new subtype constraint
 	 */
 	public <S extends F> Pattern<T, S> subtype(final Class<S> subtypeClass) {
-		if (filterFunction == null) {
-			this.filterFunction = new SubtypeFilterFunction<F>(subtypeClass);
+		if (condition == null) {
+			this.condition = new SubtypeCondition<F>(subtypeClass);
 		} else {
-			this.filterFunction = new AndFilterFunction<F>(this.filterFunction, new SubtypeFilterFunction<F>(subtypeClass));
+			this.condition = new AndCondition<>(this.condition,
+					new SubtypeCondition<F>(subtypeClass));
 		}
 
 		@SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java
index f183f0f..ae48df3 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/SubtypeFilterFunction.java
@@ -21,11 +21,15 @@ package org.apache.flink.cep.pattern;
 import org.apache.flink.api.common.functions.FilterFunction;
 
 /**
- * A filter function which filters elements of the given type. A element if filtered out iff it
+ * @deprecated This is only used when migrating from an older Flink version.
+ * Use the {@link org.apache.flink.cep.pattern.conditions.SubtypeCondition} instead.
+ *
+ * <p>A filter function which filters elements of the given type. A element if filtered out iff it
  * is not assignable to the given subtype of T.
  *
  * @param <T> Type of the elements to be filtered
  */
+@Deprecated
 public class SubtypeFilterFunction<T> implements FilterFunction<T> {
 	private static final long serialVersionUID = -2990017519957561355L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/AndCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/AndCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/AndCondition.java
new file mode 100644
index 0000000..5df7c66
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/AndCondition.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cep.pattern.conditions;
+
+/**
+ * A {@link IterativeCondition condition} which combines two conditions with a logical
+ * {@code AND} and returns {@code true} if both are {@code true}.
+ *
+ * @param <T> Type of the element to filter
+ */
+public class AndCondition<T> extends IterativeCondition<T> {
+
+	private static final long serialVersionUID = -2471892317390197319L;
+
+	private final IterativeCondition<T> left;
+	private final IterativeCondition<T> right;
+
+	public AndCondition(final IterativeCondition<T> left, final IterativeCondition<T> right) {
+		this.left = left;
+		this.right = right;
+	}
+
+	@Override
+	public boolean filter(T value, Context<T> ctx) throws Exception {
+		return left.filter(value, ctx) && right.filter(value, ctx);
+	}
+
+	/**
+	 * @return One of the {@link IterativeCondition conditions} combined in this condition.
+	 */
+	public IterativeCondition<T> getLeft() {
+		return left;
+	}
+
+	/**
+	 * @return One of the {@link IterativeCondition conditions} combined in this condition.
+	 */
+	public IterativeCondition<T> getRight() {
+		return right;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java
new file mode 100644
index 0000000..d67b407
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/BooleanConditions.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cep.pattern.conditions;
+
+/**
+ * Utility class containing an {@link IterativeCondition} that always returns
+ * {@code true} and one that always returns {@code false}.
+ */
+public class BooleanConditions {
+
+	/**
+	 * @return An {@link IterativeCondition} that always returns {@code true}.
+	 */
+	public static <T> IterativeCondition<T> trueFunction()  {
+		return new SimpleCondition<T>() {
+			private static final long serialVersionUID = 8379409657655181451L;
+
+			@Override
+			public boolean filter(T value) throws Exception {
+				return true;
+			}
+		};
+	}
+
+	/**
+	 * @return An {@link IterativeCondition} that always returns {@code false}.
+	 */
+	public static <T> IterativeCondition<T> falseFunction()  {
+		return new SimpleCondition<T>() {
+			private static final long serialVersionUID = -823981593720949910L;
+
+			@Override
+			public boolean filter(T value) throws Exception {
+				return false;
+			}
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
new file mode 100644
index 0000000..f225e01
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cep.pattern.conditions;
+
+import org.apache.flink.api.common.functions.Function;
+
+import java.io.Serializable;
+
+/**
+ * A user-defined condition that decides if an element should be accepted in the pattern or not.
+ * Accepting an element also signals a state transition for the corresponding {@link org.apache.flink.cep.nfa.NFA}.
+ *
+ * <p>A condition can be a simple filter or a more complex condition that iterates over the previously accepted
+ * elements in the pattern and decides to accept a new element or not based on some statistic over these elements.
+ * In the former case, the condition should extend the {@link SimpleCondition} class. In the later, the condition
+ * should extend this class, which gives you also access to the previously accepted elements through a {@link Context}.
+ *
+ * <p>An iterative condition that accepts an element if i) its name is middle, and ii) the sum of the prices of all
+ * accepted elements is less than {@code 5} would look like:
+ *
+ * <pre>
+ * {@code
+ * private class MyCondition extends IterativeCondition<Event> {
+ *
+ * 		@Override
+ *     	public boolean filter(Event value, Context<Event> ctx) throws Exception {
+ *     		if (!value.getName().equals("middle")) {
+ *     			return false;
+ *     		}
+ *
+ *     		double sum = 0.0;
+ *     		for (Event e: ctx.getEventsForPattern("middle")) {
+ *     			sum += e.getPrice();
+ *     		}
+ *     		sum += value.getPrice();
+ *     		return Double.compare(sum, 5.0) <= 0;
+ *     	}
+ *    }
+ * }
+ * </pre>
+ *
+ * <b>ATTENTION: </b> The call to {@link Context#getEventsForPattern(String) getEventsForPattern(...)} has to find
+ * the elements that belong to the pattern among the elements stored by the NFA. The cost of this operation can vary,
+ * so when implementing your condition, try to minimize the times the method is called.
+ */
+public abstract class IterativeCondition<T> implements Function, Serializable {
+
+	private static final long serialVersionUID = 7067817235759351255L;
+
+	/**
+	 * The filter function that evaluates the predicate.
+	 * <p>
+	 * <strong>IMPORTANT:</strong> The system assumes that the function does not
+	 * modify the elements on which the predicate is applied. Violating this assumption
+	 * can lead to incorrect results.
+	 *
+	 * @param value The value to be tested.
+	 * @param ctx The {@link Context} used for the evaluation of the function and provides access to
+	 *            the already accepted events in the pattern (see {@link Context#getEventsForPattern(String)}).
+	 * @return {@code true} for values that should be retained, {@code false}
+	 * for values to be filtered out.
+	 *
+	 * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
+	 *                   to fail and may trigger recovery.
+	 */
+	public abstract boolean filter(T value, Context<T> ctx) throws Exception;
+
+	/**
+	 * The context used when evaluating the {@link IterativeCondition condition}.
+	 */
+	public interface Context<T> extends Serializable {
+
+		/**
+		 * @return An {@link Iterable} over the already accepted elements
+		 * for a given pattern. Elements are iterated in the order the were
+		 * inserted in the pattern.
+		 *
+		 * @param name The name of the pattern.
+		 */
+		Iterable<T> getEventsForPattern(String name);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7fbdc100/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/NotCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/NotCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/NotCondition.java
new file mode 100644
index 0000000..3e6ab56
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/NotCondition.java
@@ -0,0 +1,40 @@
+/*
+ * 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.cep.pattern.conditions;
+
+/**
+ * A {@link IterativeCondition condition} which negates the condition it wraps
+ * and returns {@code true} if the original condition returns {@code false}.
+ *
+ * @param <T> Type of the element to filter
+ */
+public class NotCondition<T> extends IterativeCondition<T> {
+	private static final long serialVersionUID = -2109562093871155005L;
+
+	private final IterativeCondition<T> original;
+
+	public NotCondition(final IterativeCondition<T> original) {
+		this.original = original;
+	}
+
+	@Override
+	public boolean filter(T value, Context<T> ctx) throws Exception {
+		return !original.filter(value, ctx);
+	}
+}


[31/50] [abbrv] flink git commit: [FLINK-5911] [gelly] Command-line parameters

Posted by fh...@apache.org.
[FLINK-5911] [gelly] Command-line parameters

Create interface for parsing command-line parameters using ParameterTool
and generic implementations for boolean, long, double, string, choice.

This closes #3433


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

Branch: refs/heads/table-retraction
Commit: f2a8bc92df583c0a6beb519fd8a84cb40edbd060
Parents: 037b5cb
Author: Greg Hogan <co...@greghogan.com>
Authored: Tue Feb 28 12:35:07 2017 -0500
Committer: Greg Hogan <co...@greghogan.com>
Committed: Mon Mar 27 06:53:29 2017 -0400

----------------------------------------------------------------------
 .../drivers/parameter/BooleanParameter.java     |  54 +++
 .../drivers/parameter/ChoiceParameter.java      | 113 ++++++
 .../drivers/parameter/DoubleParameter.java      | 177 ++++++++++
 .../graph/drivers/parameter/LongParameter.java  | 129 +++++++
 .../graph/drivers/parameter/Parameter.java      |  55 +++
 .../drivers/parameter/ParameterizedBase.java    |  67 ++++
 .../drivers/parameter/SimpleParameter.java      |  72 ++++
 .../drivers/parameter/StringParameter.java      |  60 ++++
 .../flink/graph/drivers/parameter/Util.java     |  54 +++
 .../drivers/parameter/BooleanParameterTest.java |  53 +++
 .../drivers/parameter/ChoiceParameterTest.java  |  84 +++++
 .../drivers/parameter/DoubleParameterTest.java  | 341 +++++++++++++++++++
 .../drivers/parameter/LongParameterTest.java    | 223 ++++++++++++
 .../drivers/parameter/ParameterTestBase.java    |  39 +++
 .../drivers/parameter/StringParameterTest.java  |  78 +++++
 15 files changed, 1599 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java
new file mode 100644
index 0000000..7d05dbf
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Boolean}.
+ */
+public class BooleanParameter
+extends SimpleParameter<Boolean> {
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public BooleanParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	@Override
+	public String getUsage() {
+		return "[--" + name + "]";
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = parameterTool.has(name);
+	}
+
+	@Override
+	public String toString() {
+		return Boolean.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
new file mode 100644
index 0000000..b239b93
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.graph.drivers.parameter;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@link Parameter} storing a list of {@link String} choices and parsing
+ * the user's configured selection.
+ */
+public class ChoiceParameter
+extends SimpleParameter<String> {
+
+	private List<String> choices = new ArrayList<>();
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public ChoiceParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value and add to the list of choices.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public ChoiceParameter setDefaultValue(String defaultValue) {
+		super.setDefaultValue(defaultValue);
+		choices.add(defaultValue);
+		return this;
+	}
+
+	/**
+	 * Add additional choices. This function can be called multiple times.
+	 *
+	 * @param choices additional choices
+	 * @return this
+	 */
+	public ChoiceParameter addChoices(String... choices) {
+		Collections.addAll(this.choices, choices);
+		return this;
+	}
+
+	@Override
+	public String getUsage() {
+		String option = new StrBuilder()
+			.append("--")
+			.append(name)
+			.append(" <")
+			.append(StringUtils.join(choices, " | "))
+			.append(">")
+			.toString();
+
+		return hasDefaultValue ? "[" + option + "]" : option;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		Preconditions.checkArgument(choices.size() > 0, "No choices provided");
+
+		String selected = parameterTool.get(name);
+
+		if (selected == null) {
+			if (hasDefaultValue) {
+				value = defaultValue;
+				return;
+			} else {
+				throw new ProgramParametrizationException(
+					"Must select a choice for option '" + name + "': '[" + StringUtils.join(choices, ", ") + "]'");
+			}
+		}
+
+		for (String choice : choices) {
+			if (choice.equals(selected)) {
+				this.value = selected;
+				return;
+			}
+		}
+
+		throw new ProgramParametrizationException(
+			"Selection '" + selected + "' for option '" + name + "' is not in choices '[" + StringUtils.join(choices, ", ") + "]'");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
new file mode 100644
index 0000000..78753a2
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Double}.
+ */
+public class DoubleParameter
+extends SimpleParameter<Double> {
+
+	private boolean hasMinimumValue = false;
+	private boolean minimumValueInclusive;
+	private double minimumValue;
+
+	private boolean hasMaximumValue = false;
+	private boolean maximumValueInclusive;
+	private double maximumValue;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public DoubleParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value
+	 * @return this
+	 */
+	public DoubleParameter setDefaultValue(double defaultValue) {
+		super.setDefaultValue(defaultValue);
+
+		if (hasMinimumValue) {
+			if (minimumValueInclusive) {
+				Util.checkParameter(defaultValue >= minimumValue,
+					"Default value (" + defaultValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+			} else {
+				Util.checkParameter(defaultValue > minimumValue,
+					"Default value (" + defaultValue + ") must be greater than minimum (" + minimumValue + ")");
+			}
+		}
+
+		if (hasMaximumValue) {
+			if (maximumValueInclusive) {
+				Util.checkParameter(defaultValue <= maximumValue,
+					"Default value (" + defaultValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+			} else {
+				Util.checkParameter(defaultValue < maximumValue,
+					"Default value (" + defaultValue + ") must be less than maximum (" + maximumValue + ")");
+			}
+		}
+
+		return this;
+	}
+
+	/**
+	 * Set the minimum value. The minimum value is an acceptable value if and
+	 * only if inclusive is set to true.
+	 *
+	 * @param minimumValue the minimum value
+	 * @param inclusive whether the minimum value is a valid value
+	 * @return this
+	 */
+	public DoubleParameter setMinimumValue(double minimumValue, boolean inclusive) {
+		if (hasDefaultValue) {
+			if (inclusive) {
+				Util.checkParameter(minimumValue <= defaultValue,
+					"Minimum value (" + minimumValue + ") must be less than or equal to default (" + defaultValue + ")");
+			} else {
+				Util.checkParameter(minimumValue < defaultValue,
+					"Minimum value (" + minimumValue + ") must be less than default (" + defaultValue + ")");
+			}
+		} else if (hasMaximumValue) {
+			if (inclusive && maximumValueInclusive) {
+				Util.checkParameter(minimumValue <= maximumValue,
+					"Minimum value (" + minimumValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+			} else {
+				Util.checkParameter(minimumValue < maximumValue,
+					"Minimum value (" + minimumValue + ") must be less than maximum (" + maximumValue + ")");
+			}
+		}
+
+		this.hasMinimumValue = true;
+		this.minimumValue = minimumValue;
+		this.minimumValueInclusive = inclusive;
+
+		return this;
+	}
+
+	/**
+	 * Set the maximum value. The maximum value is an acceptable value if and
+	 * only if inclusive is set to true.
+	 *
+	 * @param maximumValue the maximum value
+	 * @param inclusive whether the maximum value is a valid value
+	 * @return this
+	 */
+	public DoubleParameter setMaximumValue(double maximumValue, boolean inclusive) {
+		if (hasDefaultValue) {
+			if (inclusive) {
+				Util.checkParameter(maximumValue >= defaultValue,
+					"Maximum value (" + maximumValue + ") must be greater than or equal to default (" + defaultValue + ")");
+			} else {
+				Util.checkParameter(maximumValue > defaultValue,
+					"Maximum value (" + maximumValue + ") must be greater than default (" + defaultValue + ")");
+			}
+		} else if (hasMinimumValue) {
+			if (inclusive && minimumValueInclusive) {
+				Util.checkParameter(maximumValue >= minimumValue,
+					"Maximum value (" + maximumValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+			} else {
+				Util.checkParameter(maximumValue > minimumValue,
+					"Maximum value (" + maximumValue + ") must be greater than minimum (" + minimumValue + ")");
+			}
+		}
+
+		this.hasMaximumValue = true;
+		this.maximumValue = maximumValue;
+		this.maximumValueInclusive = inclusive;
+
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.getDouble(name, defaultValue) : parameterTool.getDouble(name);
+
+		if (hasMinimumValue) {
+			if (minimumValueInclusive) {
+				Util.checkParameter(value >= minimumValue,
+					name + " must be greater than or equal to " + minimumValue);
+			} else {
+				Util.checkParameter(value > minimumValue,
+					name + " must be greater than " + minimumValue);
+			}
+		}
+
+		if (hasMaximumValue) {
+			if (maximumValueInclusive) {
+				Util.checkParameter(value <= maximumValue,
+					name + " must be less than or equal to " + maximumValue);
+			} else {
+				Util.checkParameter(value < maximumValue,
+					name + " must be less than " + maximumValue);
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return Double.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
new file mode 100644
index 0000000..5917b14
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Long}.
+ */
+public class LongParameter
+extends SimpleParameter<Long> {
+
+	private boolean hasMinimumValue = false;
+	private long minimumValue;
+
+	private boolean hasMaximumValue = false;
+	private long maximumValue;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public LongParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public LongParameter setDefaultValue(long defaultValue) {
+		super.setDefaultValue(defaultValue);
+
+		if (hasMinimumValue) {
+			Util.checkParameter(defaultValue >= minimumValue,
+				"Default value (" + defaultValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+		}
+
+		if (hasMaximumValue) {
+			Util.checkParameter(defaultValue <= maximumValue,
+				"Default value (" + defaultValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+		}
+
+		return this;
+	}
+
+	/**
+	 * Set the minimum value.
+	 *
+	 * @param minimumValue the minimum value
+	 * @return this
+	 */
+	public LongParameter setMinimumValue(long minimumValue) {
+		if (hasDefaultValue) {
+			Util.checkParameter(minimumValue <= defaultValue,
+				"Minimum value (" + minimumValue + ") must be less than or equal to default (" + defaultValue + ")");
+		} else if (hasMaximumValue) {
+			Util.checkParameter(minimumValue <= maximumValue,
+				"Minimum value (" + minimumValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+		}
+
+		this.hasMinimumValue = true;
+		this.minimumValue = minimumValue;
+
+		return this;
+	}
+
+	/**
+	 * Set the maximum value.
+	 *
+	 * @param maximumValue the maximum value
+	 * @return this
+	 */
+	public LongParameter setMaximumValue(long maximumValue) {
+		if (hasDefaultValue) {
+			Util.checkParameter(maximumValue >= defaultValue,
+				"Maximum value (" + maximumValue + ") must be greater than or equal to default (" + defaultValue + ")");
+		} else if (hasMinimumValue) {
+			Util.checkParameter(maximumValue >= minimumValue,
+				"Maximum value (" + maximumValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+		}
+
+		this.hasMaximumValue = true;
+		this.maximumValue = maximumValue;
+
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.getLong(name, defaultValue) : parameterTool.getLong(name);
+
+		if (hasMinimumValue) {
+			Util.checkParameter(value >= minimumValue,
+				name + " must be greater than or equal to " + minimumValue);
+		}
+
+		if (hasMaximumValue) {
+			Util.checkParameter(value <= maximumValue,
+				name + " must be less than or equal to " + maximumValue);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return Long.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
new file mode 100644
index 0000000..46785f8
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * Encapsulates the usage and configuration of a command-line parameter.
+ *
+ * @param <T> parameter value type
+ */
+public interface Parameter<T> {
+
+	/**
+	 * An informal usage string. Parameter names are prefixed with "--".
+	 *
+	 * Optional parameters are enclosed by "[" and "]".
+	 *
+	 * Generic values are represented by all-caps with specific values enclosed
+	 * by "&lt;" and "&gt;".
+	 *
+	 * @return command-line usage string
+	 */
+	String getUsage();
+
+	/**
+	 * Read and parse the parameter value from command-line arguments.
+	 *
+	 * @param parameterTool parameter parser
+	 */
+	void configure(ParameterTool parameterTool);
+
+	/**
+	 * Get the parameter value.
+	 *
+	 * @return parameter value
+	 */
+	T getValue();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
new file mode 100644
index 0000000..3b9b80a
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Base class for a {@link Parameterized} which maintains a list of parameters
+ * used to print the command-line usage string and configure parameters.
+ */
+public abstract class ParameterizedBase
+implements Parameterized {
+
+	private List<Parameter<?>> parameters = new ArrayList<>();
+
+	/**
+	 * Adds a parameter to the list. Parameter order is preserved when printing
+	 * the command-line usage string.
+	 *
+	 * @param parameter to add to the list of parameters
+	 */
+	public void addParameter(Parameter<?> parameter) {
+		parameters.add(parameter);
+	}
+
+	@Override
+	public String getParameterization() {
+		StrBuilder strBuilder = new StrBuilder();
+
+		// print parameters as ordered list
+		for (Parameter<?> parameter : parameters) {
+			strBuilder
+				.append(parameter.getUsage())
+				.append(" ");
+		}
+
+		return strBuilder.toString();
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) throws ProgramParametrizationException {
+		for (Parameter<?> parameter : parameters) {
+			parameter.configure(parameterTool);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
new file mode 100644
index 0000000..93469ac
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.graph.drivers.parameter;
+
+/**
+ * A {@link Parameter} with a default value.
+ */
+public abstract class SimpleParameter<T>
+implements Parameter<T> {
+
+	protected final String name;
+
+	protected boolean hasDefaultValue = false;
+
+	protected T defaultValue;
+
+	protected T value;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	protected SimpleParameter(ParameterizedBase owner, String name) {
+		this.name = name;
+		owner.addParameter(this);
+	}
+
+	/**
+	 * Set the default value, used if no value is set by the command-line
+	 * configuration.
+	 *
+	 * @param defaultValue the default value
+	 * @return this
+	 */
+	protected SimpleParameter setDefaultValue(T defaultValue) {
+		this.hasDefaultValue = true;
+		this.defaultValue = defaultValue;
+
+		return this;
+	}
+
+	@Override
+	public String getUsage() {
+		String option = "--" + name + " " + name.toUpperCase();
+
+		return hasDefaultValue ? "[" + option + "]" : option;
+	}
+
+	@Override
+	public T getValue() {
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
new file mode 100644
index 0000000..34194ec
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link String}.
+ */
+public class StringParameter
+extends SimpleParameter<String> {
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public StringParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public StringParameter setDefaultValue(String defaultValue) {
+		super.setDefaultValue(defaultValue);
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.get(name, defaultValue) : parameterTool.getRequired(name);
+	}
+
+	@Override
+	public String toString() {
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java
new file mode 100644
index 0000000..ccc290e
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.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.graph.drivers.parameter;
+
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * Utility methods for parsing command-line arguments.
+ */
+public class Util {
+
+	private Util() {}
+
+	// ------------------------------------------------------------------------
+	//  Boolean Condition Checking (Argument)
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Checks the given boolean condition, and throws an {@code ProgramParametrizationException} if
+	 * the condition is not met (evaluates to {@code false}). The exception will have the
+	 * given error message.
+	 *
+	 * @param condition The condition to check
+	 * @param errorMessage The message for the {@code ProgramParametrizationException} that is thrown if the check fails.
+	 *
+	 * @throws ProgramParametrizationException Thrown, if the condition is violated.
+	 *
+	 * @see Preconditions#checkNotNull(Object, String)
+	 */
+	public static void checkParameter(boolean condition, @Nullable Object errorMessage) {
+		if (!condition) {
+			throw new ProgramParametrizationException(String.valueOf(errorMessage));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
new file mode 100644
index 0000000..43bba88
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class BooleanParameterTest
+extends ParameterTestBase {
+
+	private BooleanParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new BooleanParameter(owner, "test");
+	}
+
+	@Test
+	public void testTrue() {
+		Assert.assertEquals("[--test]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test"}));
+		Assert.assertEquals(true, parameter.getValue());
+	}
+
+	@Test
+	public void testFalse() {
+		Assert.assertEquals("[--test]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(false, parameter.getValue());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
new file mode 100644
index 0000000..1ed1af3
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ChoiceParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private ChoiceParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+		parameter = new ChoiceParameter(owner, "choice");
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue("default").addChoices("c0", "c1", "c2");
+		Assert.assertEquals("[--choice <default | c0 | c1 | c2>]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--choice", "c1"}));
+		Assert.assertEquals("c1", parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue("default").addChoices("c0", "c1", "c2");
+		Assert.assertEquals("[--choice <default | c0 | c1 | c2>]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals("default", parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		parameter.addChoices("c0", "c1", "c2");
+		Assert.assertEquals("--choice <c0 | c1 | c2>", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--choice", "c2"}));
+		Assert.assertEquals("c2", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		parameter.addChoices("c0", "c1", "c2");
+		Assert.assertEquals("--choice <c0 | c1 | c2>", parameter.getUsage());
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Must select a choice for option 'choice': '[c0, c1, c2]'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
new file mode 100644
index 0000000..7a4d4fa
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
@@ -0,0 +1,341 @@
+/*
+ * 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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class DoubleParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private DoubleParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new DoubleParameter(owner, "test");
+	}
+
+	// Test configuration
+
+	@Test
+	public void testDefaultValueBelowMinimum() {
+		parameter.setMinimumValue(1.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0.0) must be greater than minimum (1.0)");
+
+		parameter.setDefaultValue(0.0);
+	}
+
+	@Test
+	public void testDefaultValueBetweenMinAndMax() {
+		parameter.setMinimumValue(-1.0, false);
+		parameter.setMaximumValue(1.0, false);
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueAboveMaximum() {
+		parameter.setMaximumValue(-1.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0.0) must be less than maximum (-1.0)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testMinimumValueAboveMaximum() {
+		parameter.setMaximumValue(0.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1.0) must be less than maximum (0.0)");
+
+		parameter.setMinimumValue(1.0, false);
+	}
+
+	@Test
+	public void testMinimumValueAboveDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1.0) must be less than default (0.0)");
+
+		parameter.setMinimumValue(1.0, false);
+	}
+
+	@Test
+	public void testMaximumValueBelowMinimum() {
+		parameter.setMinimumValue(0.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1.0) must be greater than minimum (0.0)");
+
+		parameter.setMaximumValue(-1.0, false);
+	}
+
+	@Test
+	public void testMaximumValueBelowDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1.0) must be greater than default (0.0)");
+
+		parameter.setMaximumValue(-1.0, false);
+	}
+
+	@Test
+	public void testEqualMinimumAndMaximumInclusive() {
+		parameter.setMinimumValue(0.0, true);
+		parameter.setMaximumValue(0.0, true);
+	}
+
+	@Test
+	public void testMinimumEqualsMaximumExclusive() {
+		parameter.setMaximumValue(0.0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (0.0) must be less than maximum (0.0)");
+
+		parameter.setMinimumValue(0.0, false);
+	}
+
+	@Test
+	public void testMaximumEqualsMinimumExclusive() {
+		parameter.setMinimumValue(0.0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (0.0) must be greater than minimum (0.0)");
+
+		parameter.setMaximumValue(0.0, false);
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue(43.21);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "12.34"}));
+		Assert.assertEquals(new Double(12.34), parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue(43.21);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(new Double(43.21), parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "12.34"}));
+		Assert.assertEquals(new Double(12.34), parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+
+	// Min
+
+	@Test
+	public void testMinInRange() {
+		parameter.setMinimumValue(0, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Double(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAtRangeInclusive() {
+		parameter.setMinimumValue(0, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAtRangeExclusive() {
+		parameter.setMinimumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+	}
+
+	@Test
+	public void testMinOutOfRange() {
+		parameter.setMinimumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	@Test
+	public void testMinOutOfRangeExclusive() {
+		parameter.setMinimumValue(0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	// Max
+
+	@Test
+	public void testMaxInRange() {
+		parameter.setMaximumValue(0, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Double(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxAtRangeInclusive() {
+		parameter.setMaximumValue(0, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxAtRangeExclusive() {
+		parameter.setMaximumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+	}
+
+	@Test
+	public void testMaxOutOfRange() {
+		parameter.setMaximumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	@Test
+	public void testMaxOutOfRangeExclusive() {
+		parameter.setMaximumValue(0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	// Min and max
+
+	@Test
+	public void testMinAndMaxBelowRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than -1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-2"}));
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMinimumExclusive() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than -1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMinimumInclusive() {
+		parameter.setMinimumValue(-1, true);
+		parameter.setMaximumValue(1, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Double(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxInRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMaximumInclusive() {
+		parameter.setMinimumValue(-1, true);
+		parameter.setMaximumValue(1, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Double(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMaximumExclusive() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	@Test
+	public void testMinAndMaxAboveRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "2"}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
new file mode 100644
index 0000000..2c26268
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class LongParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private LongParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new LongParameter(owner, "test");
+	}
+
+	// Test configuration
+
+	@Test
+	public void testDefaultValueBelowMinimum() {
+		parameter.setMinimumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0) must be greater than or equal to minimum (1)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueBetweenMinAndMax() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueAboveMaximum() {
+		parameter.setMaximumValue(-1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0) must be less than or equal to maximum (-1)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testMinimumValueAboveMaximum() {
+		parameter.setMaximumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1) must be less than or equal to maximum (0)");
+
+		parameter.setMinimumValue(1);
+	}
+
+	@Test
+	public void testMinimumValueAboveDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1) must be less than or equal to default (0)");
+
+		parameter.setMinimumValue(1);
+	}
+
+	@Test
+	public void testMaximumValueBelowMinimum() {
+		parameter.setMinimumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1) must be greater than or equal to minimum (0)");
+
+		parameter.setMaximumValue(-1);
+	}
+
+	@Test
+	public void testMaximumValueBelowDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1) must be greater than or equal to default (0)");
+
+		parameter.setMaximumValue(-1);
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue(42);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "54"}));
+		Assert.assertEquals(new Long(54), parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue(13);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(new Long(13), parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "42"}));
+		Assert.assertEquals(new Long(42), parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+
+	// Min
+
+	@Test
+	public void testMinInRange() {
+		parameter.setMinimumValue(0);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Long(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinOutOfRange() {
+		parameter.setMinimumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to 0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	// Max
+
+	@Test
+	public void testMaxInRange() {
+		parameter.setMaximumValue(0);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Long(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxOutOfRange() {
+		parameter.setMaximumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	// Min and max
+
+	@Test
+	public void testMinAndMaxBelowRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to -1");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-2"}));
+	}
+
+	@Test
+	public void testMinAndMaxInRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Long(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAboveRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 1");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "2"}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
new file mode 100644
index 0000000..a5dc0c6
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.junit.Before;
+
+public class ParameterTestBase {
+
+	protected ParameterizedBase owner;
+
+	@Before
+	public void setup() {
+		owner = new MockParameterized();
+	}
+
+	protected static class MockParameterized
+	extends ParameterizedBase {
+		@Override
+		public String getName() {
+			return MockParameterized.class.getSimpleName();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
new file mode 100644
index 0000000..496d85c
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class StringParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private StringParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new StringParameter(owner, "test");
+	}
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue("Flink");
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "Gelly"}));
+		Assert.assertEquals("Gelly", parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue("Flink");
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals("Flink", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "Gelly"}));
+		Assert.assertEquals("Gelly", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+}


[16/50] [abbrv] flink git commit: [hotfix] Remove validateRunsInMainThread from TaskExecutor to fix TaskExecutorTest

Posted by fh...@apache.org.
[hotfix] Remove validateRunsInMainThread from TaskExecutor to fix TaskExecutorTest

Currently, the TestingSerialRpcService does not play well together with the
MainThreadValidatorUtil which assumes that rpc calls are dispatched to a mailbox
from where they are picked up. In order to support the TestingSerialRpcService
we will have to extend the MainThreadValidatorUtil to allow entering the main thread
if the RpcEndpoint already runs in the context of the main thread.


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

Branch: refs/heads/table-retraction
Commit: 11fe3dc89f6b6b24fa21cc51d5e935e91634dbe5
Parents: 09164cf
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Mar 24 10:17:36 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Mar 24 10:17:36 2017 +0100

----------------------------------------------------------------------
 .../java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/11fe3dc8/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index 4883e7d..6ad0bd9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -713,8 +713,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	private void closeResourceManagerConnection(Exception cause) {
-		validateRunsInMainThread();
-
 		if (isConnectedToResourceManager()) {
 			log.info("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerId(), cause);
 
@@ -859,8 +857,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	private void closeJobManagerConnection(JobID jobId, Exception cause) {
-		validateRunsInMainThread();
-
 		log.info("Close JobManager connection for job {}.", jobId);
 
 		// 1. fail tasks running under this JobID


[49/50] [abbrv] flink git commit: [FLINK-6200] [table] Add support for unbounded event-time OVER RANGE window.

Posted by fh...@apache.org.
[FLINK-6200] [table] Add support for unbounded event-time OVER RANGE window.

This closes #3649.


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

Branch: refs/heads/table-retraction
Commit: 44f9c76a9ff50e95947c9f78a86b485f564e3796
Parents: aa3c395
Author: hongyuhong 00223286 <ho...@huawei.com>
Authored: Wed Mar 29 10:29:17 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Mar 30 22:12:21 2017 +0200

----------------------------------------------------------------------
 .../datastream/DataStreamOverAggregate.scala    |  20 +--
 .../table/runtime/aggregate/AggregateUtil.scala |  26 ++-
 .../UnboundedEventTimeOverProcessFunction.scala | 163 +++++++++++++++----
 .../table/api/scala/stream/sql/SqlITCase.scala  | 132 +++++++++++++++
 4 files changed, 292 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/44f9c76a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 7b744f1..e24dd23 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -127,14 +127,8 @@ class DataStreamOverAggregate(
         // row-time OVER window
         if (overWindow.lowerBound.isPreceding &&
               overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
-          if (overWindow.isRows) {
-            // unbounded preceding OVER ROWS window
-            createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
-          } else {
-            // unbounded preceding OVER RANGE window
-            throw new TableException(
-              "row-time OVER RANGE UNBOUNDED PRECEDING window is not supported yet.")
-          }
+          // ROWS/RANGE clause unbounded OVER window
+          createUnboundedAndCurrentRowEventTimeOverWindow(inputDS, overWindow.isRows)
         } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
           // bounded OVER window
           if (overWindow.isRows) {
@@ -202,8 +196,8 @@ class DataStreamOverAggregate(
 
   def createBoundedAndCurrentRowOverWindow(
     inputDS: DataStream[Row],
-    isRangeClause: Boolean = false,
-    isRowTimeType: Boolean = false): DataStream[Row] = {
+    isRangeClause: Boolean,
+    isRowTimeType: Boolean): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
     val partitionKeys: Array[Int] = overWindow.keys.toArray
@@ -247,7 +241,8 @@ class DataStreamOverAggregate(
   }
 
   def createUnboundedAndCurrentRowEventTimeOverWindow(
-    inputDS: DataStream[Row]): DataStream[Row]  = {
+    inputDS: DataStream[Row],
+    isRows: Boolean): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
     val partitionKeys: Array[Int] = overWindow.keys.toArray
@@ -258,7 +253,8 @@ class DataStreamOverAggregate(
 
     val processFunction = AggregateUtil.createUnboundedEventTimeOverProcessFunction(
       namedAggregates,
-      inputType)
+      inputType,
+      isRows)
 
     val result: DataStream[Row] =
       // partitioned aggregation

http://git-wip-us.apache.org/repos/asf/flink/blob/44f9c76a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index cbb2e53..93ab7b7 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -152,7 +152,8 @@ object AggregateUtil {
     */
   private[flink] def createUnboundedEventTimeOverProcessFunction(
    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-   inputType: RelDataType): UnboundedEventTimeOverProcessFunction = {
+   inputType: RelDataType,
+   isRows: Boolean): UnboundedEventTimeOverProcessFunction = {
 
     val (aggFields, aggregates) =
       transformToAggregateFunctions(
@@ -162,12 +163,23 @@ object AggregateUtil {
 
     val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
 
-    new UnboundedEventTimeOverProcessFunction(
-      aggregates,
-      aggFields,
-      inputType.getFieldCount,
-      aggregationStateType,
-      FlinkTypeFactory.toInternalRowTypeInfo(inputType))
+    if (isRows) {
+      // ROWS unbounded over process function
+      new UnboundedEventTimeRowsOverProcessFunction(
+        aggregates,
+        aggFields,
+        inputType.getFieldCount,
+        aggregationStateType,
+        FlinkTypeFactory.toInternalRowTypeInfo(inputType))
+    } else {
+      // RANGE unbounded over process function
+      new UnboundedEventTimeRangeOverProcessFunction(
+        aggregates,
+        aggFields,
+        inputType.getFieldCount,
+        aggregationStateType,
+        FlinkTypeFactory.toInternalRowTypeInfo(inputType))
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/44f9c76a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
index 7616ede..92faf7d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
@@ -41,7 +41,7 @@ import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
   * @param inputType the input row tye which the state saved
   *
   */
-class UnboundedEventTimeOverProcessFunction(
+abstract class UnboundedEventTimeOverProcessFunction(
     private val aggregates: Array[AggregateFunction[_]],
     private val aggFields: Array[Int],
     private val forwardedFieldCount: Int,
@@ -53,7 +53,7 @@ class UnboundedEventTimeOverProcessFunction(
   Preconditions.checkNotNull(aggFields)
   Preconditions.checkArgument(aggregates.length == aggFields.length)
 
-  private var output: Row = _
+  protected var output: Row = _
   // state to hold the accumulators of the aggregations
   private var accumulatorState: ValueState[Row] = _
   // state to hold rows until the next watermark arrives
@@ -162,30 +162,9 @@ class UnboundedEventTimeOverProcessFunction(
         val curRowList = rowMapState.get(curTimestamp)
         collector.setAbsoluteTimestamp(curTimestamp)
 
-        var j = 0
-        while (j < curRowList.size) {
-          val curRow = curRowList.get(j)
-          i = 0
-
-          // copy forwarded fields to output row
-          while (i < forwardedFieldCount) {
-            output.setField(i, curRow.getField(i))
-            i += 1
-          }
-
-          // update accumulators and copy aggregates to output row
-          i = 0
-          while (i < aggregates.length) {
-            val index = forwardedFieldCount + i
-            val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator]
-            aggregates(i).accumulate(accumulator, curRow.getField(aggFields(i)))
-            output.setField(index, aggregates(i).getValue(accumulator))
-            i += 1
-          }
-          // emit output row
-          collector.collect(output)
-          j += 1
-        }
+        // process the same timestamp datas, the mechanism is different according ROWS or RANGE
+        processElementsWithSameTimestamp(curRowList, lastAccumulator, collector)
+
         rowMapState.remove(curTimestamp)
       }
 
@@ -204,21 +183,145 @@ class UnboundedEventTimeOverProcessFunction(
    * If timestamps arrive in order (as in case of using the RocksDB state backend) this is just
    * an append with O(1).
    */
-  private def insertToSortedList(recordTimeStamp: Long) = {
+  private def insertToSortedList(recordTimestamp: Long) = {
     val listIterator = sortedTimestamps.listIterator(sortedTimestamps.size)
     var continue = true
     while (listIterator.hasPrevious && continue) {
       val timestamp = listIterator.previous
-      if (recordTimeStamp >= timestamp) {
+      if (recordTimestamp >= timestamp) {
         listIterator.next
-        listIterator.add(recordTimeStamp)
+        listIterator.add(recordTimestamp)
         continue = false
       }
     }
 
     if (continue) {
-      sortedTimestamps.addFirst(recordTimeStamp)
+      sortedTimestamps.addFirst(recordTimestamp)
     }
   }
 
+  /**
+   * Process the same timestamp datas, the mechanism is different between
+   * rows and range window.
+   */
+  def processElementsWithSameTimestamp(
+    curRowList: JList[Row],
+    lastAccumulator: Row,
+    out: Collector[Row]): Unit
+
+}
+
+/**
+  * A ProcessFunction to support unbounded ROWS window.
+  * The ROWS clause defines on a physical level how many rows are included in a window frame.
+  */
+class UnboundedEventTimeRowsOverProcessFunction(
+   aggregates: Array[AggregateFunction[_]],
+   aggFields: Array[Int],
+   forwardedFieldCount: Int,
+   intermediateType: TypeInformation[Row],
+   inputType: TypeInformation[Row])
+  extends UnboundedEventTimeOverProcessFunction(
+    aggregates,
+    aggFields,
+    forwardedFieldCount,
+    intermediateType,
+    inputType) {
+
+  override def processElementsWithSameTimestamp(
+    curRowList: JList[Row],
+    lastAccumulator: Row,
+    out: Collector[Row]): Unit = {
+
+    var j = 0
+    var i = 0
+    while (j < curRowList.size) {
+      val curRow = curRowList.get(j)
+      i = 0
+
+      // copy forwarded fields to output row
+      while (i < forwardedFieldCount) {
+        output.setField(i, curRow.getField(i))
+        i += 1
+      }
+
+      // update accumulators and copy aggregates to output row
+      i = 0
+      while (i < aggregates.length) {
+        val index = forwardedFieldCount + i
+        val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator]
+        aggregates(i).accumulate(accumulator, curRow.getField(aggFields(i)))
+        output.setField(index, aggregates(i).getValue(accumulator))
+        i += 1
+      }
+      // emit output row
+      out.collect(output)
+      j += 1
+    }
+  }
+}
+
+
+/**
+  * A ProcessFunction to support unbounded RANGE window.
+  * The RANGE option includes all the rows within the window frame
+  * that have the same ORDER BY values as the current row.
+  */
+class UnboundedEventTimeRangeOverProcessFunction(
+    aggregates: Array[AggregateFunction[_]],
+    aggFields: Array[Int],
+    forwardedFieldCount: Int,
+    intermediateType: TypeInformation[Row],
+    inputType: TypeInformation[Row])
+  extends UnboundedEventTimeOverProcessFunction(
+    aggregates,
+    aggFields,
+    forwardedFieldCount,
+    intermediateType,
+    inputType) {
+
+  override def processElementsWithSameTimestamp(
+    curRowList: JList[Row],
+    lastAccumulator: Row,
+    out: Collector[Row]): Unit = {
+
+    var j = 0
+    var i = 0
+    // all same timestamp data should have same aggregation value.
+    while (j < curRowList.size) {
+      val curRow = curRowList.get(j)
+      i = 0
+      while (i < aggregates.length) {
+        val index = forwardedFieldCount + i
+        val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator]
+        aggregates(i).accumulate(accumulator, curRow.getField(aggFields(i)))
+        i += 1
+      }
+      j += 1
+    }
+
+    // emit output row
+    j = 0
+    while (j < curRowList.size) {
+      val curRow = curRowList.get(j)
+
+      // copy forwarded fields to output row
+      i = 0
+      while (i < forwardedFieldCount) {
+        output.setField(i, curRow.getField(i))
+        i += 1
+      }
+
+      //copy aggregates to output row
+      i = 0
+      while (i < aggregates.length) {
+        val index = forwardedFieldCount + i
+        val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator]
+        output.setField(index, aggregates(i).getValue(accumulator))
+        i += 1
+      }
+      out.collect(output)
+      j += 1
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/44f9c76a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index b8285a1..0d3a46c 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -840,6 +840,138 @@ class SqlITCase extends StreamingWithStateTestBase {
       "6,8,Hello world,51,9,5,9,1")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
+
+  /** test sliding event-time non-partitioned unbounded RANGE window **/
+  @Test
+  def testUnboundedNonPartitionedEventTimeRangeWindow(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 1L, "Hello")),
+      Left(14000002L, (1, 1L, "Hello")),
+      Left(14000002L, (1, 2L, "Hello")),
+      Left(14000002L, (1, 3L, "Hello world")),
+      Left(14000003L, (2, 2L, "Hello world")),
+      Left(14000003L, (2, 3L, "Hello world")),
+      Right(14000020L),
+      Left(14000021L, (1, 4L, "Hello world")),
+      Left(14000022L, (1, 5L, "Hello world")),
+      Left(14000022L, (1, 6L, "Hello world")),
+      Left(14000022L, (1, 7L, "Hello world")),
+      Left(14000023L, (2, 4L, "Hello world")),
+      Left(14000023L, (2, 5L, "Hello world")),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "2,1,Hello,1,1,1,1,1",
+      "1,1,Hello,7,4,1,3,1",
+      "1,2,Hello,7,4,1,3,1",
+      "1,3,Hello world,7,4,1,3,1",
+      "2,2,Hello world,12,6,2,3,1",
+      "2,3,Hello world,12,6,2,3,1",
+      "1,1,Hi,13,7,1,3,1",
+      "1,4,Hello world,17,8,2,4,1",
+      "1,5,Hello world,35,11,3,7,1",
+      "1,6,Hello world,35,11,3,7,1",
+      "1,7,Hello world,35,11,3,7,1",
+      "2,4,Hello world,44,13,3,7,1",
+      "2,5,Hello world,44,13,3,7,1"
+    )
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test sliding event-time unbounded RANGE window **/
+  @Test
+  def testUnboundedPartitionedEventTimeRangeWindow(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 1L, "Hello")),
+      Left(14000002L, (1, 1L, "Hello")),
+      Left(14000002L, (1, 2L, "Hello")),
+      Left(14000002L, (1, 3L, "Hello world")),
+      Left(14000003L, (2, 2L, "Hello world")),
+      Left(14000003L, (2, 3L, "Hello world")),
+      Right(14000020L),
+      Left(14000021L, (1, 4L, "Hello world")),
+      Left(14000022L, (1, 5L, "Hello world")),
+      Left(14000022L, (1, 6L, "Hello world")),
+      Left(14000022L, (1, 7L, "Hello world")),
+      Left(14000023L, (2, 4L, "Hello world")),
+      Left(14000023L, (2, 5L, "Hello world")),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,1,Hello,6,3,2,3,1",
+      "1,2,Hello,6,3,2,3,1",
+      "1,3,Hello world,6,3,2,3,1",
+      "1,1,Hi,7,4,1,3,1",
+      "2,1,Hello,1,1,1,1,1",
+      "2,2,Hello world,6,3,2,3,1",
+      "2,3,Hello world,6,3,2,3,1",
+      "1,4,Hello world,11,5,2,4,1",
+      "1,5,Hello world,29,8,3,7,1",
+      "1,6,Hello world,29,8,3,7,1",
+      "1,7,Hello world,29,8,3,7,1",
+      "2,4,Hello world,15,5,3,5,1",
+      "2,5,Hello world,15,5,3,5,1"
+    )
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
 }
 
 object SqlITCase {


[18/50] [abbrv] flink git commit: [FLINK-5890] [gelly] GatherSumApply broken when object reuse enabled

Posted by fh...@apache.org.
[FLINK-5890] [gelly] GatherSumApply broken when object reuse enabled

The initial fix for this ticket is not working on larger data sets.

Reduce supports returning the left input, right input, a new object, or
a locally reused object. The trouble with the initial fix was that the
returned local object was reusing fields from the input tuples.

The problem is with ReduceDriver#run managing two values (reuse1 and
reuse2) and with a third, local value returned by
GatherSumApplyIteration.SumUDF. After the first grouping value.f1 ==
reuse1.f1. Following UDF calls may swap value.f1 and reuse2.f1, which
causes reuse1.f1 == reuse2.f1. With an odd number of swaps the next
grouping will reduce with reuse1 and reuse2 sharing a field and
deserialization will overwrite stored values.

The simple fix is to only use and return the provided inputs.

This closes #3515


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

Branch: refs/heads/table-retraction
Commit: 524b20f2db70fc4afba3a539fbf249c6d768ab4f
Parents: 4b19e27
Author: Greg Hogan <co...@greghogan.com>
Authored: Fri Mar 10 16:44:27 2017 -0500
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 24 11:03:12 2017 -0400

----------------------------------------------------------------------
 .../org/apache/flink/graph/gsa/GatherSumApplyIteration.java     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/524b20f2/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
index e941b7b..5c07a73 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
@@ -330,7 +330,6 @@ public class GatherSumApplyIteration<K, VV, EV, M> implements CustomUnaryOperati
 
 		@Override
 		public Tuple2<K, M> reduce(Tuple2<K, M> arg0, Tuple2<K, M> arg1) throws Exception {
-			K key = arg0.f0;
 			M result = this.sumFunction.sum(arg0.f1, arg1.f1);
 
 			// if the user returns value from the right argument then swap as
@@ -339,9 +338,11 @@ public class GatherSumApplyIteration<K, VV, EV, M> implements CustomUnaryOperati
 				M tmp = arg1.f1;
 				arg1.f1 = arg0.f1;
 				arg0.f1 = tmp;
+			} else {
+				arg0.f1 = result;
 			}
 
-			return new Tuple2<>(key, result);
+			return arg0;
 		}
 
 		@Override


[42/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Add safety check for execution graph state transitions

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Add safety check for execution graph state transitions


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

Branch: refs/heads/table-retraction
Commit: 60895a3ccd83609088be6ecef3445f7c78c9955a
Parents: 874d956
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 22 19:53:50 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 17:11:49 2017 +0200

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionGraph.java  |  8 +++++++
 .../flink/runtime/jobgraph/JobStatus.java       | 22 +++++++++++++++++++-
 2 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/60895a3c/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index e911f49..1c7b1c8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -1105,6 +1105,14 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	}
 
 	private boolean transitionState(JobStatus current, JobStatus newState, Throwable error) {
+		// consistency check
+		if (current.isTerminalState()) {
+			String message = "Job is trying to leave terminal state " + current;
+			LOG.error(message);
+			throw new IllegalStateException(message);
+		}
+
+		// now do the actual state transition
 		if (STATE_UPDATER.compareAndSet(this, current, newState)) {
 			LOG.info("Job {} ({}) switched from state {} to {}.", getJobName(), getJobID(), current, newState, error);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/60895a3c/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
index 6a0ac97..4ef86bd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
@@ -69,11 +69,31 @@ public enum JobStatus {
 	JobStatus(TerminalState terminalState) {
 		this.terminalState = terminalState;
 	}
-	
+
+	/**
+	 * Checks whether this state is <i>globally terminal</i>. A globally terminal job
+	 * is complete and cannot fail any more and will not be restarted or recovered by another
+	 * standby master node.
+	 * 
+	 * <p>When a globally terminal state has been reached, all recovery data for the job is
+	 * dropped from the high-availability services.
+	 * 
+	 * @return True, if this job status is globally terminal, false otherwise.
+	 */
 	public boolean isGloballyTerminalState() {
 		return terminalState == TerminalState.GLOBALLY;
 	}
 
+	/**
+	 * Checks whether this state is <i>locally terminal</i>. Locally terminal refers to the
+	 * state of a job's execution graph within an executing JobManager. If the execution graph
+	 * is locally terminal, the JobManager will not continue executing or recovering the job. 
+	 *
+	 * <p>The only state that is locally terminal, but not globally terminal is {@link #SUSPENDED},
+	 * which is typically entered when the executing JobManager looses its leader status.
+	 * 
+	 * @return True, if this job status is terminal, false otherwise.
+	 */
 	public boolean isTerminalState() {
 		return terminalState != TerminalState.NON_TERMINAL;
 	}


[50/50] [abbrv] flink git commit: [FLINK-5653] [table] Add processing-time OVER ROWS BETWEEN x PRECEDING aggregation to SQL.

Posted by fh...@apache.org.
[FLINK-5653] [table] Add processing-time OVER ROWS BETWEEN x PRECEDING aggregation to SQL.

This closes #3653.
This closes #3574.


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

Branch: refs/heads/table-retraction
Commit: ee033c903b20d7a233009764b6b96e78eea5b981
Parents: 44f9c76
Author: Stefano Bortoli <s....@gmail.com>
Authored: Thu Mar 30 11:28:41 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Mar 30 22:12:21 2017 +0200

----------------------------------------------------------------------
 .../datastream/DataStreamOverAggregate.scala    |   6 +-
 .../table/runtime/aggregate/AggregateUtil.scala |   9 +-
 ...oundedProcessingOverRowProcessFunction.scala | 181 ++++++++++++++++++
 .../table/api/scala/stream/sql/SqlITCase.scala  | 184 ++++++++++++++++++-
 .../scala/stream/sql/WindowAggregateTest.scala  |  55 ++++++
 5 files changed, 423 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ee033c90/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index e24dd23..2df4e02 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -112,8 +112,10 @@ class DataStreamOverAggregate(
           // bounded OVER window
           if (overWindow.isRows) {
             // ROWS clause bounded OVER window
-            throw new TableException(
-              "processing-time OVER ROWS PRECEDING window is not supported yet.")
+            createBoundedAndCurrentRowOverWindow(
+              inputDS,
+              isRangeClause = true,
+              isRowTimeType = false)
           } else {
             // RANGE clause bounded OVER window
             throw new TableException(

http://git-wip-us.apache.org/repos/asf/flink/blob/ee033c90/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index 93ab7b7..88e9d68 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -138,8 +138,13 @@ object AggregateUtil {
         )
       }
     } else {
-      throw TableException(
-        "Bounded partitioned proc-time OVER aggregation is not supported yet.")
+      new BoundedProcessingOverRowProcessFunction(
+        aggregates,
+        aggFields,
+        precedingOffset,
+        inputType.getFieldCount,
+        aggregationStateType,
+        FlinkTypeFactory.toInternalRowTypeInfo(inputType))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ee033c90/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala
new file mode 100644
index 0000000..454b177
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRowProcessFunction.scala
@@ -0,0 +1,181 @@
+/*
+ * 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.table.runtime.aggregate
+
+import java.util
+
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{Collector, Preconditions}
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+
+class BoundedProcessingOverRowProcessFunction(
+  private val aggregates: Array[AggregateFunction[_]],
+  private val aggFields: Array[Int],
+  private val precedingOffset: Long,
+  private val forwardedFieldCount: Int,
+  private val aggregatesTypeInfo: RowTypeInfo,
+  private val inputType: TypeInformation[Row])
+    extends ProcessFunction[Row, Row] {
+
+  Preconditions.checkNotNull(aggregates)
+  Preconditions.checkNotNull(aggFields)
+  Preconditions.checkArgument(aggregates.length == aggFields.length)
+  Preconditions.checkArgument(precedingOffset > 0)
+
+  private var accumulatorState: ValueState[Row] = _
+  private var rowMapState: MapState[Long, JList[Row]] = _
+  private var output: Row = _
+  private var counterState: ValueState[Long] = _
+  private var smallestTsState: ValueState[Long] = _
+
+  override def open(config: Configuration) {
+
+    output = new Row(forwardedFieldCount + aggregates.length)
+    // We keep the elements received in a Map state keyed
+    // by the ingestion time in the operator.
+    // we also keep counter of processed elements
+    // and timestamp of oldest element
+    val rowListTypeInfo: TypeInformation[JList[Row]] =
+      new ListTypeInfo[Row](inputType).asInstanceOf[TypeInformation[JList[Row]]]
+
+    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+      new MapStateDescriptor[Long, JList[Row]]("windowBufferMapState",
+        BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
+    rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+    val aggregationStateDescriptor: ValueStateDescriptor[Row] =
+      new ValueStateDescriptor[Row]("aggregationState", aggregatesTypeInfo)
+    accumulatorState = getRuntimeContext.getState(aggregationStateDescriptor)
+
+    val processedCountDescriptor : ValueStateDescriptor[Long] =
+       new ValueStateDescriptor[Long]("processedCountState", classOf[Long])
+    counterState = getRuntimeContext.getState(processedCountDescriptor)
+
+    val smallestTimestampDescriptor : ValueStateDescriptor[Long] =
+       new ValueStateDescriptor[Long]("smallestTSState", classOf[Long])
+    smallestTsState = getRuntimeContext.getState(smallestTimestampDescriptor)
+  }
+
+  override def processElement(
+    input: Row,
+    ctx: ProcessFunction[Row, Row]#Context,
+    out: Collector[Row]): Unit = {
+
+    val currentTime = ctx.timerService.currentProcessingTime
+    var i = 0
+
+    // initialize state for the processed element
+    var accumulators = accumulatorState.value
+    if (accumulators == null) {
+      accumulators = new Row(aggregates.length)
+      while (i < aggregates.length) {
+        accumulators.setField(i, aggregates(i).createAccumulator())
+        i += 1
+      }
+    }
+
+    // get smallest timestamp
+    var smallestTs = smallestTsState.value
+    if (smallestTs == 0L) {
+      smallestTs = currentTime
+      smallestTsState.update(smallestTs)
+    }
+    // get previous counter value
+    var counter = counterState.value
+
+    if (counter == precedingOffset) {
+      val retractList = rowMapState.get(smallestTs)
+
+      // get oldest element beyond buffer size
+      // and if oldest element exist, retract value
+      i = 0
+      while (i < aggregates.length) {
+        val accumulator = accumulators.getField(i).asInstanceOf[Accumulator]
+        aggregates(i).retract(accumulator, retractList.get(0).getField(aggFields(i)))
+        i += 1
+      }
+      retractList.remove(0)
+      // if reference timestamp list not empty, keep the list
+      if (!retractList.isEmpty) {
+        rowMapState.put(smallestTs, retractList)
+      } // if smallest timestamp list is empty, remove and find new smallest
+      else {
+        rowMapState.remove(smallestTs)
+        val iter = rowMapState.keys.iterator
+        var currentTs: Long = 0L
+        var newSmallestTs: Long = Long.MaxValue
+        while (iter.hasNext) {
+          currentTs = iter.next
+          if (currentTs < newSmallestTs) {
+            newSmallestTs = currentTs
+          }
+        }
+        smallestTsState.update(newSmallestTs)
+      }
+    } // we update the counter only while buffer is getting filled
+    else {
+      counter += 1
+      counterState.update(counter)
+    }
+
+    // copy forwarded fields in output row
+    i = 0
+    while (i < forwardedFieldCount) {
+      output.setField(i, input.getField(i))
+      i += 1
+    }
+
+    // accumulate current row and set aggregate in output row
+    i = 0
+    while (i < aggregates.length) {
+      val index = forwardedFieldCount + i
+      val accumulator = accumulators.getField(i).asInstanceOf[Accumulator]
+      aggregates(i).accumulate(accumulator, input.getField(aggFields(i)))
+      output.setField(index, aggregates(i).getValue(accumulator))
+      i += 1
+    }
+
+    // update map state, accumulator state, counter and timestamp
+    val currentTimeState = rowMapState.get(currentTime)
+    if (currentTimeState != null) {
+      currentTimeState.add(input)
+      rowMapState.put(currentTime, currentTimeState)
+    } else { // add new input
+      val newList = new util.ArrayList[Row]
+      newList.add(input)
+      rowMapState.put(currentTime, newList)
+    }
+
+    accumulatorState.update(accumulators)
+
+    out.collect(output)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ee033c90/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 0d3a46c..67d13b0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -344,7 +344,7 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val expected = mutable.MutableList(
       "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3",
-      "Hello,2,3,4", "Hello,2,3,5","Hello,2,3,6",
+      "Hello,2,3,4", "Hello,2,3,5", "Hello,2,3,6",
       "Hello,3,3,7", "Hello,4,3,9", "Hello,5,3,12",
       "Hello,6,3,15",
       "Hello World,7,1,7", "Hello World,7,2,14", "Hello World,7,3,21",
@@ -471,12 +471,12 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val expected = mutable.MutableList(
       "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
-      "Hello,2,6,9", "Hello,3,6,9","Hello,2,6,9",
+      "Hello,2,6,9", "Hello,3,6,9", "Hello,2,6,9",
       "Hello,3,4,9",
       "Hello,4,2,7",
       "Hello,5,2,9",
-      "Hello,6,2,11","Hello,65,2,12",
-      "Hello,9,2,12","Hello,9,2,12","Hello,18,3,18",
+      "Hello,6,2,11", "Hello,65,2,12",
+      "Hello,9,2,12", "Hello,9,2,12", "Hello,18,3,18",
       "Hello World,7,1,7", "Hello World,17,3,21", "Hello World,77,3,21", "Hello World,18,1,7",
       "Hello World,8,2,15",
       "Hello World,20,1,20")
@@ -543,12 +543,12 @@ class SqlITCase extends StreamingWithStateTestBase {
 
     val expected = mutable.MutableList(
       "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
-      "Hello,2,6,9", "Hello,3,6,9","Hello,2,6,9",
+      "Hello,2,6,9", "Hello,3,6,9", "Hello,2,6,9",
       "Hello,3,4,9",
       "Hello,4,2,7",
       "Hello,5,2,9",
-      "Hello,6,2,11","Hello,65,2,12",
-      "Hello,9,2,12","Hello,9,2,12","Hello,18,3,18",
+      "Hello,6,2,11", "Hello,65,2,12",
+      "Hello,9,2,12", "Hello,9,2,12", "Hello,18,3,18",
       "Hello World,7,4,25", "Hello World,17,3,21", "Hello World,77,3,21", "Hello World,18,1,7",
       "Hello World,8,2,15",
       "Hello World,20,1,20")
@@ -556,7 +556,7 @@ class SqlITCase extends StreamingWithStateTestBase {
   }
 
   /**
-    *  All aggregates must be computed on the same window.
+    * All aggregates must be computed on the same window.
     */
   @Test(expected = classOf[TableException])
   def testMultiWindow(): Unit = {
@@ -972,6 +972,174 @@ class SqlITCase extends StreamingWithStateTestBase {
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
+  @Test
+  def testPartitionedProcTimeOverWindow(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(1)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery = "SELECT a,  " +
+      " SUM(c) OVER (" +
+      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " MIN(c) OVER (" +
+      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
+      " FROM MyTable"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,0,0",
+      "2,1,1",
+      "2,3,1",
+      "3,3,3",
+      "3,7,3",
+      "3,12,3",
+      "4,6,6",
+      "4,13,6",
+      "4,21,6",
+      "4,24,7",
+      "5,10,10",
+      "5,21,10",
+      "5,33,10",
+      "5,36,11",
+      "5,39,12")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testPartitionedProcTimeOverWindow2(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(1)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery = "SELECT a,  " +
+      " SUM(c) OVER (" +
+      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " MIN(c) OVER (" +
+      " PARTITION BY a ORDER BY procTime() ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) AS minC " +
+      " FROM MyTable"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,0,0",
+      "2,1,1",
+      "2,3,1",
+      "3,3,3",
+      "3,7,3",
+      "3,12,3",
+      "4,6,6",
+      "4,13,6",
+      "4,21,6",
+      "4,30,6",
+      "5,10,10",
+      "5,21,10",
+      "5,33,10",
+      "5,46,10",
+      "5,60,10")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+
+  @Test
+  def testNonPartitionedProcTimeOverWindow(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(1)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery = "SELECT a,  " +
+      " SUM(c) OVER (" +
+      " ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " MIN(c) OVER (" +
+      " ORDER BY procTime() ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS minC " +
+      " FROM MyTable"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,0,0",
+      "2,1,0",
+      "2,3,0",
+      "3,6,1",
+      "3,9,2",
+      "3,12,3",
+      "4,15,4",
+      "4,18,5",
+      "4,21,6",
+      "4,24,7",
+      "5,27,8",
+      "5,30,9",
+      "5,33,10",
+      "5,36,11",
+      "5,39,12")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testNonPartitionedProcTimeOverWindow2(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(1)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val t = StreamTestData.get5TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c, 'd, 'e)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery = "SELECT a,  " +
+      " SUM(c) OVER (" +
+      " ORDER BY procTime() ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS sumC , " +
+      " MIN(c) OVER (" +
+      " ORDER BY procTime() ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) AS minC " +
+      " FROM MyTable"
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,0,0",
+      "2,1,0",
+      "2,3,0",
+      "3,6,0",
+      "3,10,0",
+      "3,15,0",
+      "4,21,0",
+      "4,28,0",
+      "4,36,0",
+      "4,45,0",
+      "5,55,0",
+      "5,66,1",
+      "5,77,2",
+      "5,88,3",
+      "5,99,4")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
 }
 
 object SqlITCase {

http://git-wip-us.apache.org/repos/asf/flink/blob/ee033c90/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index 45d204a..52fd5f8 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -405,4 +405,59 @@ class WindowAggregateTest extends TableTestBase {
     streamUtil.verifySql(sql, expected)
   }
 
+ @Test
+  def testBoundNonPartitionedProcTimeWindowWithRowRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
+      "CURRENT ROW) as cnt1 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "PROCTIME() AS $2")
+          ),
+          term("orderBy", "PROCTIME"),
+          term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+  
+  @Test
+  def testBoundPartitionedProcTimeWindowWithRowRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 preceding AND " +
+      "CURRENT ROW) as cnt1 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "PROCTIME() AS $2")
+          ),
+          term("partitionBy", "c"),
+          term("orderBy", "PROCTIME"),
+          term("rows", "BETWEEN 2 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "PROCTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
 }


[29/50] [abbrv] flink git commit: [hotfix] Make GC test more strict in WindowOperatorContractTest

Posted by fh...@apache.org.
[hotfix] Make GC test more strict in WindowOperatorContractTest


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

Branch: refs/heads/table-retraction
Commit: 662ed33d8f5baed95035b8176daf95a1caa0b278
Parents: fad201b
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sat Mar 25 16:59:31 2017 +0100
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Sat Mar 25 16:59:31 2017 +0100

----------------------------------------------------------------------
 .../windowing/WindowOperatorContractTest.java   | 21 ++++++++++++++++++--
 1 file changed, 19 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/662ed33d/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
index faab505..3ae8f37 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -1853,13 +1853,20 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 	}
 
 	private void testGarbageCollectionTimer(TimeDomainAdaptor timeAdaptor) throws Exception {
+		long allowedLateness = 20L;
+
+		if (timeAdaptor instanceof ProcessingTimeAdaptor) {
+			// we don't have allowed lateness for processing time
+			allowedLateness = 0;
+		}
+
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
 		timeAdaptor.setIsEventTime(mockAssigner);
 		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, allowedLateness, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1879,7 +1886,17 @@ public abstract class WindowOperatorContractTest extends TestLogger {
 
 		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
 
-		timeAdaptor.advanceTime(testHarness, 19 + 20); // 19 is maxTime of the window
+		// verify that we can still fire on the GC timer
+		timeAdaptor.shouldFireOnTime(mockTrigger);
+
+		timeAdaptor.advanceTime(testHarness, 19 + allowedLateness); // 19 is maxTime of the window
+
+		// ensure that our trigger is still called
+		timeAdaptor.verifyTriggerCallback(mockTrigger, times(1), 19L + allowedLateness, null);
+
+		// ensure that our window function is called a last timer if the trigger
+		// fires on the GC timer
+		verify(mockWindowFunction, times(1)).process(eq(0), eq(new TimeWindow(0, 20)), anyInternalWindowContext(), intIterable(0), WindowOperatorContractTest.<Void>anyCollector());
 
 		verify(mockTrigger, times(1)).clear(anyTimeWindow(), anyTriggerContext());
 


[47/50] [abbrv] flink git commit: [FLINK-5655] [table] Add event-time OVER RANGE BETWEEN x PRECEDING aggregation to SQL.

Posted by fh...@apache.org.
[FLINK-5655] [table] Add event-time OVER RANGE BETWEEN x PRECEDING aggregation to SQL.

This closes #3629.


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

Branch: refs/heads/table-retraction
Commit: d4665a00a4262f89b166895f73a54daab2f25e1c
Parents: ca68110
Author: \u91d1\u7af9 <ji...@alibaba-inc.com>
Authored: Tue Mar 28 12:36:03 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Mar 30 09:25:17 2017 +0200

----------------------------------------------------------------------
 .../datastream/DataStreamOverAggregate.scala    |  17 +-
 .../table/runtime/aggregate/AggregateUtil.scala |  34 ++-
 .../RangeClauseBoundedOverProcessFunction.scala | 221 +++++++++++++++++++
 .../table/api/scala/stream/sql/SqlITCase.scala  | 144 ++++++++++++
 .../scala/stream/sql/WindowAggregateTest.scala  |  55 +++++
 5 files changed, 455 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d4665a00/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 01e5a9a..7b744f1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -139,11 +139,16 @@ class DataStreamOverAggregate(
           // bounded OVER window
           if (overWindow.isRows) {
             // ROWS clause bounded OVER window
-            createRowsClauseBoundedAndCurrentRowOverWindow(inputDS, isRowTimeType = true)
+            createBoundedAndCurrentRowOverWindow(
+              inputDS,
+              isRangeClause = false,
+              isRowTimeType = true)
           } else {
             // RANGE clause bounded OVER window
-            throw new TableException(
-              "row-time OVER RANGE PRECEDING window is not supported yet.")
+            createBoundedAndCurrentRowOverWindow(
+              inputDS,
+              isRangeClause = true,
+              isRowTimeType = true)
           }
         } else {
           throw new TableException(
@@ -195,8 +200,9 @@ class DataStreamOverAggregate(
     result
   }
 
-  def createRowsClauseBoundedAndCurrentRowOverWindow(
+  def createBoundedAndCurrentRowOverWindow(
     inputDS: DataStream[Row],
+    isRangeClause: Boolean = false,
     isRowTimeType: Boolean = false): DataStream[Row] = {
 
     val overWindow: Group = logicWindow.groups.get(0)
@@ -209,10 +215,11 @@ class DataStreamOverAggregate(
     // get the output types
     val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
 
-    val processFunction = AggregateUtil.createRowsClauseBoundedOverProcessFunction(
+    val processFunction = AggregateUtil.createBoundedOverProcessFunction(
       namedAggregates,
       inputType,
       precedingOffset,
+      isRangeClause,
       isRowTimeType
     )
     val result: DataStream[Row] =

http://git-wip-us.apache.org/repos/asf/flink/blob/d4665a00/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index fdac692..cbb2e53 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -91,20 +91,21 @@ object AggregateUtil {
   }
 
   /**
-    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for ROWS clause
+    * Create an [[org.apache.flink.streaming.api.functions.ProcessFunction]] for
     * bounded OVER window to evaluate final aggregate value.
     *
     * @param namedAggregates List of calls to aggregate functions and their output field names
     * @param inputType       Input row type
-    * @param inputFields     All input fields
     * @param precedingOffset the preceding offset
+    * @param isRangeClause   It is a tag that indicates whether the OVER clause is rangeClause
     * @param isRowTimeType   It is a tag that indicates whether the time type is rowTimeType
     * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
     */
-  private[flink] def createRowsClauseBoundedOverProcessFunction(
+  private[flink] def createBoundedOverProcessFunction(
     namedAggregates: Seq[CalcitePair[AggregateCall, String]],
     inputType: RelDataType,
     precedingOffset: Long,
+    isRangeClause: Boolean,
     isRowTimeType: Boolean): ProcessFunction[Row, Row] = {
 
     val (aggFields, aggregates) =
@@ -117,14 +118,25 @@ object AggregateUtil {
     val inputRowType = FlinkTypeFactory.toInternalRowTypeInfo(inputType).asInstanceOf[RowTypeInfo]
 
     if (isRowTimeType) {
-      new RowsClauseBoundedOverProcessFunction(
-        aggregates,
-        aggFields,
-        inputType.getFieldCount,
-        aggregationStateType,
-        inputRowType,
-        precedingOffset
-      )
+      if (isRangeClause) {
+        new RangeClauseBoundedOverProcessFunction(
+          aggregates,
+          aggFields,
+          inputType.getFieldCount,
+          aggregationStateType,
+          inputRowType,
+          precedingOffset
+        )
+      } else {
+        new RowsClauseBoundedOverProcessFunction(
+          aggregates,
+          aggFields,
+          inputType.getFieldCount,
+          aggregationStateType,
+          inputRowType,
+          precedingOffset
+        )
+      }
     } else {
       throw TableException(
         "Bounded partitioned proc-time OVER aggregation is not supported yet.")

http://git-wip-us.apache.org/repos/asf/flink/blob/d4665a00/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RangeClauseBoundedOverProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RangeClauseBoundedOverProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RangeClauseBoundedOverProcessFunction.scala
new file mode 100644
index 0000000..0c8555b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RangeClauseBoundedOverProcessFunction.scala
@@ -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.table.runtime.aggregate
+
+import java.util.{List => JList, ArrayList => JArrayList}
+
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
+import org.apache.flink.types.Row
+import org.apache.flink.util.{Collector, Preconditions}
+
+/**
+ * Process Function for RANGE clause event-time bounded OVER window
+ *
+ * @param aggregates           the list of all [[AggregateFunction]] used for this aggregation
+ * @param aggFields            the position (in the input Row) of the input value for each aggregate
+ * @param forwardedFieldCount  the count of forwarded fields.
+ * @param aggregationStateType the row type info of aggregation
+ * @param inputRowType         the row type info of input row
+ * @param precedingOffset      the preceding offset
+ */
+class RangeClauseBoundedOverProcessFunction(
+    private val aggregates: Array[AggregateFunction[_]],
+    private val aggFields: Array[Int],
+    private val forwardedFieldCount: Int,
+    private val aggregationStateType: RowTypeInfo,
+    private val inputRowType: RowTypeInfo,
+    private val precedingOffset: Long)
+  extends ProcessFunction[Row, Row] {
+
+  Preconditions.checkNotNull(aggregates)
+  Preconditions.checkNotNull(aggFields)
+  Preconditions.checkArgument(aggregates.length == aggFields.length)
+  Preconditions.checkNotNull(forwardedFieldCount)
+  Preconditions.checkNotNull(aggregationStateType)
+  Preconditions.checkNotNull(precedingOffset)
+
+  private var output: Row = _
+
+  // the state which keeps the last triggering timestamp
+  private var lastTriggeringTsState: ValueState[Long] = _
+
+  // the state which used to materialize the accumulator for incremental calculation
+  private var accumulatorState: ValueState[Row] = _
+
+  // the state which keeps all the data that are not expired.
+  // The first element (as the mapState key) of the tuple is the time stamp. Per each time stamp,
+  // the second element of tuple is a list that contains the entire data of all the rows belonging
+  // to this time stamp.
+  private var dataState: MapState[Long, JList[Row]] = _
+
+  override def open(config: Configuration) {
+
+    output = new Row(forwardedFieldCount + aggregates.length)
+
+    val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+      new ValueStateDescriptor[Long]("lastTriggeringTsState", classOf[Long])
+    lastTriggeringTsState = getRuntimeContext.getState(lastTriggeringTsDescriptor)
+
+    val accumulatorStateDescriptor =
+      new ValueStateDescriptor[Row]("accumulatorState", aggregationStateType)
+    accumulatorState = getRuntimeContext.getState(accumulatorStateDescriptor)
+
+    val keyTypeInformation: TypeInformation[Long] =
+      BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+    val valueTypeInformation: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputRowType)
+
+    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+      new MapStateDescriptor[Long, JList[Row]](
+        "dataState",
+        keyTypeInformation,
+        valueTypeInformation)
+
+    dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+  }
+
+  override def processElement(
+    input: Row,
+    ctx: ProcessFunction[Row, Row]#Context,
+    out: Collector[Row]): Unit = {
+
+    // triggering timestamp for trigger calculation
+    val triggeringTs = ctx.timestamp
+
+    val lastTriggeringTs = lastTriggeringTsState.value
+
+    // check if the data is expired, if not, save the data and register event time timer
+    if (triggeringTs > lastTriggeringTs) {
+      val data = dataState.get(triggeringTs)
+      if (null != data) {
+        data.add(input)
+        dataState.put(triggeringTs, data)
+      } else {
+        val data = new JArrayList[Row]
+        data.add(input)
+        dataState.put(triggeringTs, data)
+        // register event time timer
+        ctx.timerService.registerEventTimeTimer(triggeringTs)
+      }
+    }
+  }
+
+  override def onTimer(
+    timestamp: Long,
+    ctx: ProcessFunction[Row, Row]#OnTimerContext,
+    out: Collector[Row]): Unit = {
+    // gets all window data from state for the calculation
+    val inputs: JList[Row] = dataState.get(timestamp)
+
+    if (null != inputs) {
+
+      var accumulators = accumulatorState.value
+      var dataListIndex = 0
+      var aggregatesIndex = 0
+
+      // initialize when first run or failover recovery per key
+      if (null == accumulators) {
+        accumulators = new Row(aggregates.length)
+        aggregatesIndex = 0
+        while (aggregatesIndex < aggregates.length) {
+          accumulators.setField(aggregatesIndex, aggregates(aggregatesIndex).createAccumulator())
+          aggregatesIndex += 1
+        }
+      }
+
+      // keep up timestamps of retract data
+      val retractTsList: JList[Long] = new JArrayList[Long]
+
+      // do retraction
+      val dataTimestampIt = dataState.keys.iterator
+      while (dataTimestampIt.hasNext) {
+        val dataTs: Long = dataTimestampIt.next()
+        val offset = timestamp - dataTs
+        if (offset > precedingOffset) {
+          val retractDataList = dataState.get(dataTs)
+          dataListIndex = 0
+          while (dataListIndex < retractDataList.size()) {
+            aggregatesIndex = 0
+            while (aggregatesIndex < aggregates.length) {
+              val accumulator = accumulators.getField(aggregatesIndex).asInstanceOf[Accumulator]
+              aggregates(aggregatesIndex)
+                .retract(accumulator, retractDataList.get(dataListIndex)
+                .getField(aggFields(aggregatesIndex)))
+              aggregatesIndex += 1
+            }
+            dataListIndex += 1
+          }
+          retractTsList.add(dataTs)
+        }
+      }
+
+      // do accumulation
+      dataListIndex = 0
+      while (dataListIndex < inputs.size()) {
+        // accumulate current row
+        aggregatesIndex = 0
+        while (aggregatesIndex < aggregates.length) {
+          val accumulator = accumulators.getField(aggregatesIndex).asInstanceOf[Accumulator]
+          aggregates(aggregatesIndex).accumulate(accumulator, inputs.get(dataListIndex)
+            .getField(aggFields(aggregatesIndex)))
+          aggregatesIndex += 1
+        }
+        dataListIndex += 1
+      }
+
+      // set aggregate in output row
+      aggregatesIndex = 0
+      while (aggregatesIndex < aggregates.length) {
+        val index = forwardedFieldCount + aggregatesIndex
+        val accumulator = accumulators.getField(aggregatesIndex).asInstanceOf[Accumulator]
+        output.setField(index, aggregates(aggregatesIndex).getValue(accumulator))
+        aggregatesIndex += 1
+      }
+
+      // copy forwarded fields to output row and emit output row
+      dataListIndex = 0
+      while (dataListIndex < inputs.size()) {
+        aggregatesIndex = 0
+        while (aggregatesIndex < forwardedFieldCount) {
+          output.setField(aggregatesIndex, inputs.get(dataListIndex).getField(aggregatesIndex))
+          aggregatesIndex += 1
+        }
+        out.collect(output)
+        dataListIndex += 1
+      }
+
+      // remove the data that has been retracted
+      dataListIndex = 0
+      while (dataListIndex < retractTsList.size) {
+        dataState.remove(retractTsList.get(dataListIndex))
+        dataListIndex += 1
+      }
+
+      // update state
+      accumulatorState.update(accumulators)
+      lastTriggeringTsState.update(timestamp)
+    }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/d4665a00/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 80ff42a..b8285a1 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -411,6 +411,150 @@ class SqlITCase extends StreamingWithStateTestBase {
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
+  @Test
+  def testBoundPartitionedEventTimeWindowWithRange(): Unit = {
+    val data = Seq(
+      Left((1500L, (1L, 15, "Hello"))),
+      Left((1600L, (1L, 16, "Hello"))),
+      Left((1000L, (1L, 1, "Hello"))),
+      Left((2000L, (2L, 2, "Hello"))),
+      Right(1000L),
+      Left((2000L, (2L, 2, "Hello"))),
+      Left((2000L, (2L, 3, "Hello"))),
+      Left((3000L, (3L, 3, "Hello"))),
+      Right(2000L),
+      Left((4000L, (4L, 4, "Hello"))),
+      Right(3000L),
+      Left((5000L, (5L, 5, "Hello"))),
+      Right(5000L),
+      Left((6000L, (6L, 6, "Hello"))),
+      Left((6500L, (6L, 65, "Hello"))),
+      Right(7000L),
+      Left((9000L, (6L, 9, "Hello"))),
+      Left((9500L, (6L, 18, "Hello"))),
+      Left((9000L, (6L, 9, "Hello"))),
+      Right(10000L),
+      Left((10000L, (7L, 7, "Hello World"))),
+      Left((11000L, (7L, 17, "Hello World"))),
+      Left((11000L, (7L, 77, "Hello World"))),
+      Right(12000L),
+      Left((14000L, (7L, 18, "Hello World"))),
+      Right(14000L),
+      Left((15000L, (8L, 8, "Hello World"))),
+      Right(17000L),
+      Left((20000L, (20L, 20, "Hello World"))),
+      Right(19000L))
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t1 = env
+      .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val sqlQuery = "SELECT " +
+      "c, b, " +
+      "count(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      "preceding AND CURRENT ROW)" +
+      ", sum(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      " preceding AND CURRENT ROW)" +
+      " from T1"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
+      "Hello,2,6,9", "Hello,3,6,9","Hello,2,6,9",
+      "Hello,3,4,9",
+      "Hello,4,2,7",
+      "Hello,5,2,9",
+      "Hello,6,2,11","Hello,65,2,12",
+      "Hello,9,2,12","Hello,9,2,12","Hello,18,3,18",
+      "Hello World,7,1,7", "Hello World,17,3,21", "Hello World,77,3,21", "Hello World,18,1,7",
+      "Hello World,8,2,15",
+      "Hello World,20,1,20")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testBoundNonPartitionedEventTimeWindowWithRange(): Unit = {
+    val data = Seq(
+      Left((1500L, (1L, 15, "Hello"))),
+      Left((1600L, (1L, 16, "Hello"))),
+      Left((1000L, (1L, 1, "Hello"))),
+      Left((2000L, (2L, 2, "Hello"))),
+      Right(1000L),
+      Left((2000L, (2L, 2, "Hello"))),
+      Left((2000L, (2L, 3, "Hello"))),
+      Left((3000L, (3L, 3, "Hello"))),
+      Right(2000L),
+      Left((4000L, (4L, 4, "Hello"))),
+      Right(3000L),
+      Left((5000L, (5L, 5, "Hello"))),
+      Right(5000L),
+      Left((6000L, (6L, 6, "Hello"))),
+      Left((6500L, (6L, 65, "Hello"))),
+      Right(7000L),
+      Left((9000L, (6L, 9, "Hello"))),
+      Left((9500L, (6L, 18, "Hello"))),
+      Left((9000L, (6L, 9, "Hello"))),
+      Right(10000L),
+      Left((10000L, (7L, 7, "Hello World"))),
+      Left((11000L, (7L, 17, "Hello World"))),
+      Left((11000L, (7L, 77, "Hello World"))),
+      Right(12000L),
+      Left((14000L, (7L, 18, "Hello World"))),
+      Right(14000L),
+      Left((15000L, (8L, 8, "Hello World"))),
+      Right(17000L),
+      Left((20000L, (20L, 20, "Hello World"))),
+      Right(19000L))
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t1 = env
+      .addSource[(Long, Int, String)](new EventTimeSourceFunction[(Long, Int, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val sqlQuery = "SELECT " +
+      "c, b, " +
+      "count(a) OVER (ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      "preceding AND CURRENT ROW)" +
+      ", sum(a) OVER (ORDER BY RowTime() RANGE BETWEEN INTERVAL '1' SECOND " +
+      " preceding AND CURRENT ROW)" +
+      " from T1"
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3",
+      "Hello,2,6,9", "Hello,3,6,9","Hello,2,6,9",
+      "Hello,3,4,9",
+      "Hello,4,2,7",
+      "Hello,5,2,9",
+      "Hello,6,2,11","Hello,65,2,12",
+      "Hello,9,2,12","Hello,9,2,12","Hello,18,3,18",
+      "Hello World,7,4,25", "Hello World,17,3,21", "Hello World,77,3,21", "Hello World,18,1,7",
+      "Hello World,8,2,15",
+      "Hello World,20,1,20")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
   /**
     *  All aggregates must be computed on the same window.
     */

http://git-wip-us.apache.org/repos/asf/flink/blob/d4665a00/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index 7b8b2df..45d204a 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -350,4 +350,59 @@ class WindowAggregateTest extends TableTestBase {
     streamUtil.verifySql(sql, expected)
   }
 
+  @Test
+  def testBoundPartitionedRowTimeWindowWithRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (PARTITION BY c ORDER BY RowTime() " +
+      "RANGE BETWEEN INTERVAL '1' SECOND  preceding AND CURRENT ROW) as cnt1 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("partitionBy", "c"),
+          term("orderBy", "ROWTIME"),
+          term("range", "BETWEEN 1000 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
+  @Test
+  def testBoundNonPartitionedRowTimeWindowWithRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (ORDER BY RowTime() " +
+      "RANGE BETWEEN INTERVAL '1' SECOND  preceding AND CURRENT ROW) as cnt1 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("orderBy", "ROWTIME"),
+          term("range", "BETWEEN 1000 PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0")
+        ),
+        term("select", "c", "w0$o0 AS $1")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
 }


[44/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Move metrics out of the Execution Graph

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Move metrics out of the Execution Graph

ExecutionGraph-based metrics should be in their own package 'org.apache.flink.runtime.executiongraph.metrics'.
They are instantiated by whoever builds the execution graph, but not by the execution graph itself.
This separates concerns more elegantly.


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

Branch: refs/heads/table-retraction
Commit: c277ee17388c85c40d9c3956fe9ac524c3157130
Parents: 85f75a5
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 29 16:54:17 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 17:11:49 2017 +0200

----------------------------------------------------------------------
 .../runtime/executiongraph/ExecutionGraph.java  |  45 +----
 .../executiongraph/ExecutionGraphBuilder.java   |   8 +-
 .../metrics/RestartTimeGauge.java               |  80 +++++++++
 .../executiongraph/metrics/package-info.java    |  23 +++
 ...ExecutionGraphCheckpointCoordinatorTest.java |   4 +-
 .../ExecutionGraphMetricsTest.java              | 173 ++++++-------------
 .../partitioner/RescalePartitionerTest.java     |   5 +-
 7 files changed, 169 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 0564fd0..06b2f9a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -29,9 +29,6 @@ import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.StoppingException;
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
@@ -126,8 +123,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 	/** The log object used for debugging. */
 	static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class);
 
-	static final String RESTARTING_TIME_METRIC_NAME = "restartingTime";
-
 	// --------------------------------------------------------------------------------------------
 
 	/** The lock used to secure all access to mutable fields, especially the tracking of progress
@@ -258,9 +253,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			Collections.<BlobKey>emptyList(),
 			Collections.<URL>emptyList(),
 			slotProvider,
-			ExecutionGraph.class.getClassLoader(),
-			new UnregisteredMetricsGroup()
-		);
+			ExecutionGraph.class.getClassLoader());
 	}
 
 	public ExecutionGraph(
@@ -275,8 +268,7 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 			List<BlobKey> requiredJarFiles,
 			List<URL> requiredClasspaths,
 			SlotProvider slotProvider,
-			ClassLoader userClassLoader,
-			MetricGroup metricGroup) throws IOException {
+			ClassLoader userClassLoader) throws IOException {
 
 		checkNotNull(futureExecutor);
 		checkNotNull(jobId);
@@ -315,9 +307,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		this.scheduleAllocationTimeout = checkNotNull(timeout);
 
 		this.restartStrategy = restartStrategy;
-
-		metricGroup.gauge(RESTARTING_TIME_METRIC_NAME, new RestartTimeGauge());
-
 		this.kvStateLocationRegistry = new KvStateLocationRegistry(jobId, getAllVertices());
 	}
 
@@ -1449,36 +1438,6 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 		}
 	}
 
-	/**
-	 * Gauge which returns the last restarting time. Restarting time is the time between
-	 * JobStatus.RESTARTING and JobStatus.RUNNING or a terminal state if JobStatus.RUNNING was not
-	 * reached. If the job has not yet reached either of these states, then the time is measured
-	 * since reaching JobStatus.RESTARTING. If it is still the initial job execution, then the
-	 * gauge will return 0.
-	 */
-	private class RestartTimeGauge implements Gauge<Long> {
-
-		@Override
-		public Long getValue() {
-			long restartingTimestamp = stateTimestamps[JobStatus.RESTARTING.ordinal()];
-
-			if (restartingTimestamp <= 0) {
-				// we haven't yet restarted our job
-				return 0L;
-			} else if (stateTimestamps[JobStatus.RUNNING.ordinal()] >= restartingTimestamp) {
-				// we have transitioned to RUNNING since the last restart
-				return stateTimestamps[JobStatus.RUNNING.ordinal()] - restartingTimestamp;
-			} else if (state.isTerminalState()) {
-				// since the last restart we've switched to a terminal state without touching
-				// the RUNNING state (e.g. failing from RESTARTING)
-				return stateTimestamps[state.ordinal()] - restartingTimestamp;
-			} else {
-				// we're still somwhere between RESTARTING and RUNNING
-				return System.currentTimeMillis() - restartingTimestamp;
-			}
-		}
-	}
-
 	@Override
 	public ArchivedExecutionGraph archive() {
 		Map<JobVertexID, ArchivedExecutionJobVertex> archivedTasks = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
index 8471178..494b7a2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -102,8 +103,7 @@ public class ExecutionGraphBuilder {
 						jobGraph.getUserJarBlobKeys(),
 						jobGraph.getClasspaths(),
 						slotProvider,
-						classLoader,
-						metrics);
+						classLoader);
 		} catch (IOException e) {
 			throw new JobException("Could not create the execution graph.", e);
 		}
@@ -250,6 +250,10 @@ public class ExecutionGraphBuilder {
 					checkpointStatsTracker);
 		}
 
+		// create all the metrics for the Execution Graph
+
+		metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph));
+
 		return executionGraph;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java
new file mode 100644
index 0000000..e0a22e3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java
@@ -0,0 +1,80 @@
+/*
+ * 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.runtime.executiongraph.metrics;
+
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Gauge which returns the last restarting time.
+ * 
+ * <p>Restarting time is the time between {@link JobStatus#RESTARTING} and {@link JobStatus#RUNNING},
+ * or a terminal state if {@link JobStatus#RUNNING} was not reached.
+ * 
+ * <p>If the job has not yet reached either of these states, then the time is measured since reaching
+ * {@link JobStatus#RESTARTING}. If it is still the initial job execution, then the gauge will return 0.
+ */
+public class RestartTimeGauge implements Gauge<Long> {
+
+	public static final String METRIC_NAME = "restartingTime";
+
+	// ------------------------------------------------------------------------
+
+	private final ExecutionGraph eg;
+
+	public RestartTimeGauge(ExecutionGraph executionGraph) {
+		this.eg = checkNotNull(executionGraph);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public Long getValue() {
+		final JobStatus status = eg.getState();
+
+		final long restartingTimestamp = eg.getStatusTimestamp(JobStatus.RESTARTING);
+
+		final long switchToRunningTimestamp;
+		final long lastRestartTime;
+
+		if (restartingTimestamp <= 0) {
+			// we haven't yet restarted our job
+			return 0L;
+		}
+		else if ((switchToRunningTimestamp = eg.getStatusTimestamp(JobStatus.RUNNING)) >= restartingTimestamp) {
+			// we have transitioned to RUNNING since the last restart
+			lastRestartTime = switchToRunningTimestamp - restartingTimestamp;
+		}
+		else if (status.isTerminalState()) {
+			// since the last restart we've switched to a terminal state without touching
+			// the RUNNING state (e.g. failing from RESTARTING)
+			lastRestartTime = eg.getStatusTimestamp(status) - restartingTimestamp;
+		}
+		else {
+			// we're still somewhere between RESTARTING and RUNNING
+			lastRestartTime  = System.currentTimeMillis() - restartingTimestamp;
+		}
+
+		// we guard this with 'Math.max' to avoid negative timestamps when clocks re-sync 
+		return Math.max(lastRestartTime, 0);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/package-info.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/package-info.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/package-info.java
new file mode 100644
index 0000000..8b9d205
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains classes implementing various metrics for the job execution,
+ * based on the {@link org.apache.flink.runtime.executiongraph.ExecutionGraph Execution Graph}.
+ */
+package org.apache.flink.runtime.executiongraph.metrics;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
index 81162b6..0ab031e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -95,8 +94,7 @@ public class ExecutionGraphCheckpointCoordinatorTest {
 			Collections.<BlobKey>emptyList(),
 			Collections.<URL>emptyList(),
 			new Scheduler(TestingUtils.defaultExecutionContext()),
-			ClassLoader.getSystemClassLoader(),
-			new UnregisteredMetricsGroup());
+			ClassLoader.getSystemClassLoader());
 
 		executionGraph.enableCheckpointing(
 				100,

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
index 5496e35..97127c7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
@@ -20,13 +20,7 @@ package org.apache.flink.runtime.executiongraph;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.Metric;
-import org.apache.flink.metrics.MetricConfig;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.metrics.reporter.MetricReporter;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.clusterframework.types.AllocationID;
@@ -35,13 +29,9 @@ import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
+import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
-import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.instance.Slot;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -49,9 +39,11 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
-import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot;
+import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
@@ -63,17 +55,16 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class ExecutionGraphMetricsTest extends TestLogger {
 
@@ -86,32 +77,14 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 		try {
 			// setup execution graph with mocked scheduling logic
 			int parallelism = 1;
-	
+
 			JobVertex jobVertex = new JobVertex("TestVertex");
 			jobVertex.setParallelism(parallelism);
 			jobVertex.setInvokableClass(NoOpInvokable.class);
 			JobGraph jobGraph = new JobGraph("Test Job", jobVertex);
-	
-			Configuration config = new Configuration();
-			config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test");
-			config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName());
-	
+
 			Configuration jobConfig = new Configuration();
-	
 			Time timeout = Time.seconds(10L);
-	
-			MetricRegistry metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config));
-	
-			assertTrue(metricRegistry.getReporters().size() == 1);
-	
-			MetricReporter reporter = metricRegistry.getReporters().get(0);
-	
-			assertTrue(reporter instanceof TestingReporter);
-	
-			TestingReporter testingReporter = (TestingReporter) reporter;
-	
-			MetricGroup metricGroup = new JobManagerMetricGroup(metricRegistry, "localhost");
-	
 			Scheduler scheduler = mock(Scheduler.class);
 
 			ResourceID taskManagerId = ResourceID.generate();
@@ -163,163 +136,127 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 				Collections.<BlobKey>emptyList(),
 				Collections.<URL>emptyList(),
 				scheduler,
-				getClass().getClassLoader(),
-				metricGroup);
-	
-			// get restarting time metric
-			Metric metric = testingReporter.getMetric(ExecutionGraph.RESTARTING_TIME_METRIC_NAME);
-	
-			assertNotNull(metric);
-			assertTrue(metric instanceof Gauge);
-	
-			@SuppressWarnings("unchecked")
-			Gauge<Long> restartingTime = (Gauge<Long>) metric;
-	
+				getClass().getClassLoader());
+
+			RestartTimeGauge restartingTime = new RestartTimeGauge(executionGraph);
+
 			// check that the restarting time is 0 since it's the initial start
-			assertTrue(0L == restartingTime.getValue());
-	
+			assertEquals(0L, restartingTime.getValue().longValue());
+
 			executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources());
-	
+
 			// start execution
-		executionGraph.scheduleForExecution();
+			executionGraph.scheduleForExecution();
+			assertEquals(0L, restartingTime.getValue().longValue());
 
-			assertTrue(0L == restartingTime.getValue());
-	
 			List<ExecutionAttemptID> executionIDs = new ArrayList<>();
-	
+
 			for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) {
 				executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId());
 			}
-	
+
 			// tell execution graph that the tasks are in state running --> job status switches to state running
 			for (ExecutionAttemptID executionID : executionIDs) {
 				executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING));
 			}
-	
+
 			assertEquals(JobStatus.RUNNING, executionGraph.getState());
-	
-			assertTrue(0L == restartingTime.getValue());
-	
+			assertEquals(0L, restartingTime.getValue().longValue());
+
 			// fail the job so that it goes into state restarting
 			for (ExecutionAttemptID executionID : executionIDs) {
 				executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception()));
 			}
-	
+
 			assertEquals(JobStatus.RESTARTING, executionGraph.getState());
-	
+
 			long firstRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING);
-	
+
 			// wait some time so that the restarting time gauge shows a value different from 0
 			Thread.sleep(50);
-	
+
 			long previousRestartingTime = restartingTime.getValue();
-	
+
 			// check that the restarting time is monotonically increasing
 			for (int i = 0; i < 10; i++) {
 				long currentRestartingTime = restartingTime.getValue();
-	
+
 				assertTrue(currentRestartingTime >= previousRestartingTime);
 				previousRestartingTime = currentRestartingTime;
 			}
-	
+
 			// check that we have measured some restarting time
 			assertTrue(previousRestartingTime > 0);
-	
+
 			// restart job
 			testingRestartStrategy.restartExecutionGraph();
-	
+
 			executionIDs.clear();
-	
+
 			for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) {
 				executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId());
 			}
-	
+
 			for (ExecutionAttemptID executionID : executionIDs) {
 				executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING));
 			}
-	
+
 			assertEquals(JobStatus.RUNNING, executionGraph.getState());
-	
+
 			assertTrue(firstRestartingTimestamp != 0);
-	
+
 			previousRestartingTime = restartingTime.getValue();
-	
+
 			// check that the restarting time does not increase after we've reached the running state
 			for (int i = 0; i < 10; i++) {
 				long currentRestartingTime = restartingTime.getValue();
-	
+
 				assertTrue(currentRestartingTime == previousRestartingTime);
 				previousRestartingTime = currentRestartingTime;
 			}
-	
+
 			// fail job again
 			for (ExecutionAttemptID executionID : executionIDs) {
 				executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception()));
 			}
-	
+
 			assertEquals(JobStatus.RESTARTING, executionGraph.getState());
-	
+
 			long secondRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING);
-	
+
 			assertTrue(firstRestartingTimestamp != secondRestartingTimestamp);
-	
+
 			Thread.sleep(50);
-	
+
 			previousRestartingTime = restartingTime.getValue();
-	
+
 			// check that the restarting time is increasing again
 			for (int i = 0; i < 10; i++) {
 				long currentRestartingTime = restartingTime.getValue();
-	
+
 				assertTrue(currentRestartingTime >= previousRestartingTime);
 				previousRestartingTime = currentRestartingTime;
 			}
-	
+
 			assertTrue(previousRestartingTime > 0);
-	
+
 			// now lets fail the job while it is in restarting and see whether the restarting time then stops to increase
 			// for this to work, we have to use a SuppressRestartException
 			executionGraph.fail(new SuppressRestartsException(new Exception()));
-	
+
 			assertEquals(JobStatus.FAILED, executionGraph.getState());
-	
+
 			previousRestartingTime = restartingTime.getValue();
-	
+
 			for (int i = 0; i < 10; i++) {
 				long currentRestartingTime = restartingTime.getValue();
-	
+
 				assertTrue(currentRestartingTime == previousRestartingTime);
 				previousRestartingTime = currentRestartingTime;
 			}
 		} finally {
 			executor.shutdownNow();
 		}
-
-	}
-
-	public static class TestingReporter implements MetricReporter {
-
-		private final Map<String, Metric> metrics = new HashMap<>();
-
-		@Override
-		public void open(MetricConfig config) {}
-
-		@Override
-		public void close() {}
-
-		@Override
-		public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) {
-			metrics.put(metricName, metric);
-		}
-
-		@Override
-		public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) {
-			metrics.remove(metricName);
-		}
-
-		Metric getMetric(String metricName) {
-			return metrics.get(metricName);
-		}
 	}
 
 	static class TestingRestartStrategy implements RestartStrategy {

http://git-wip-us.apache.org/repos/asf/flink/blob/c277ee17/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java
index 43fe169..d72c37b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java
@@ -23,7 +23,6 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.blob.BlobKey;
@@ -44,6 +43,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Before;
 import org.junit.Test;
 
@@ -150,8 +150,7 @@ public class RescalePartitionerTest extends TestLogger {
 			new ArrayList<BlobKey>(),
 			new ArrayList<URL>(),
 			new Scheduler(TestingUtils.defaultExecutionContext()),
-			ExecutionGraph.class.getClassLoader(),
-			new UnregisteredMetricsGroup());
+			ExecutionGraph.class.getClassLoader());
 		try {
 			eg.attachJobGraph(jobVertices);
 		}


[43/50] [abbrv] flink git commit: [hotfix] [dist. coordination] Fix waiting for execution termination

Posted by fh...@apache.org.
[hotfix] [dist. coordination] Fix waiting for execution termination


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

Branch: refs/heads/table-retraction
Commit: 85f75a5999cfeabb82f24f1f3c4cd998a4c5b348
Parents: 60895a3
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Mar 24 10:45:07 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 17:11:49 2017 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/executiongraph/ExecutionGraph.java   | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/85f75a59/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 1c7b1c8..0564fd0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -1090,11 +1090,10 @@ public class ExecutionGraph implements AccessExecutionGraph, Archiveable<Archive
 
 	/**
 	 * For testing: This waits until the job execution has finished.
-	 * @throws InterruptedException
 	 */
 	public void waitUntilFinished() throws InterruptedException {
 		synchronized (progressLock) {
-			while (!state.isGloballyTerminalState()) {
+			while (!state.isTerminalState()) {
 				progressLock.wait();
 			}
 		}


[45/50] [abbrv] flink git commit: [FLINK-5340] [metrics] Add an uptime and downtime metric to the Execution Graph.

Posted by fh...@apache.org.
[FLINK-5340] [metrics] Add an uptime and downtime metric to the Execution Graph.


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

Branch: refs/heads/table-retraction
Commit: 719d0cf19664556e62f808469fb641127c3f4410
Parents: 231bec8
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Mar 29 18:24:13 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Mar 29 21:52:47 2017 +0200

----------------------------------------------------------------------
 .../executiongraph/ExecutionGraphBuilder.java   |  4 +
 .../executiongraph/metrics/DownTimeGauge.java   | 79 ++++++++++++++++++++
 .../executiongraph/metrics/UpTimeGauge.java     | 67 +++++++++++++++++
 3 files changed, 150 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/719d0cf1/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
index f1da8bd..a6455f5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -33,7 +33,9 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
 import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.executiongraph.metrics.DownTimeGauge;
 import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge;
+import org.apache.flink.runtime.executiongraph.metrics.UpTimeGauge;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -247,6 +249,8 @@ public class ExecutionGraphBuilder {
 		// create all the metrics for the Execution Graph
 
 		metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph));
+		metrics.gauge(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph));
+		metrics.gauge(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph));
 
 		return executionGraph;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/719d0cf1/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java
new file mode 100644
index 0000000..5f24587
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.executiongraph.metrics;
+
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A gauge that returns (in milliseconds) how long a job has not been not running any
+ * more, in case it is in a failing/recovering situation. Running jobs return naturally
+ * a value of zero.
+ * 
+ * <p>For jobs that have never run (new not yet scheduled jobs), this gauge returns
+ * {@value NOT_YET_RUNNING}, and for jobs that are not running any more, it returns
+ * {@value NO_LONGER_RUNNING}. 
+ */
+public class DownTimeGauge implements Gauge<Long> {
+
+	public static final String METRIC_NAME = "downtime";
+
+	private static final long NOT_YET_RUNNING = 0L;
+
+	private static final long NO_LONGER_RUNNING = -1L;
+
+	// ------------------------------------------------------------------------
+
+	private final ExecutionGraph eg;
+
+	public DownTimeGauge(ExecutionGraph executionGraph) {
+		this.eg = checkNotNull(executionGraph);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public Long getValue() {
+		final JobStatus status = eg.getState();
+
+		if (status == JobStatus.RUNNING) {
+			// running right now - no downtime
+			return 0L;
+		}
+		else if (status.isTerminalState()) {
+			// not running any more -> finished or not on leader
+			return NO_LONGER_RUNNING;
+		}
+		else {
+			final long runningTimestamp = eg.getStatusTimestamp(JobStatus.RUNNING);
+			if (runningTimestamp > 0) {
+				// job was running at some point and is not running now
+				// we use 'Math.max' here to avoid negative timestamps when clocks change
+				return Math.max(System.currentTimeMillis() - runningTimestamp, 0);
+			}
+			else {
+				// job was never scheduled so far
+				return NOT_YET_RUNNING;
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/719d0cf1/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java
new file mode 100644
index 0000000..d3f6224
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.executiongraph.metrics;
+
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A gauge that returns (in milliseconds) how long a job has been running.
+ * 
+ * <p>For jobs that are not running any more, it returns {@value NO_LONGER_RUNNING}. 
+ */
+public class UpTimeGauge implements Gauge<Long> {
+
+	public static final String METRIC_NAME = "uptime";
+
+	private static final long NO_LONGER_RUNNING = -1L;
+
+	// ------------------------------------------------------------------------
+
+	private final ExecutionGraph eg;
+
+	public UpTimeGauge(ExecutionGraph executionGraph) {
+		this.eg = checkNotNull(executionGraph);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public Long getValue() {
+		final JobStatus status = eg.getState();
+
+		if (status == JobStatus.RUNNING) {
+			// running right now - report the uptime
+			final long runningTimestamp = eg.getStatusTimestamp(JobStatus.RUNNING);
+			// we use 'Math.max' here to avoid negative timestamps when clocks change
+			return Math.max(System.currentTimeMillis() - runningTimestamp, 0);
+		}
+		else if (status.isTerminalState()) {
+			// not running any more -> finished or not on leader
+			return NO_LONGER_RUNNING;
+		}
+		else {
+			// not yet running or not up at the moment
+			return 0L;
+		}
+	}
+}
\ No newline at end of file


[08/50] [abbrv] flink git commit: [FLINK-4354] [heartbeat] Implement heartbeat logic between TaskManager and ResourceManager

Posted by fh...@apache.org.
[FLINK-4354] [heartbeat] Implement heartbeat logic between TaskManager and ResourceManager

This closes #3591.


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

Branch: refs/heads/table-retraction
Commit: fd90672f9ccf7a0e02e5eb9c6251dc3d451ce8ba
Parents: d20fb09
Author: Zhijiang <wa...@aliyun.com>
Authored: Wed Mar 22 15:12:33 2017 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Mar 23 13:58:44 2017 +0100

----------------------------------------------------------------------
 .../heartbeat/TestingHeartbeatServices.java     |  52 ++++++++
 .../flink/runtime/jobmaster/JobMaster.java      |   4 +-
 .../flink/runtime/minicluster/MiniCluster.java  |   5 +-
 .../RegistrationConnectionListener.java         |  40 ++++++
 .../resourcemanager/ResourceManager.java        | 117 ++++++++++++++----
 .../resourcemanager/ResourceManagerGateway.java |  16 +++
 .../resourcemanager/ResourceManagerRunner.java  |   9 +-
 .../StandaloneResourceManager.java              |   5 +
 .../runtime/taskexecutor/TaskExecutor.java      | 106 ++++++++++++++--
 .../taskexecutor/TaskExecutorGateway.java       |  14 +++
 .../TaskExecutorRegistrationSuccess.java        |  16 ++-
 ...TaskExecutorToResourceManagerConnection.java |  21 +++-
 .../clusterframework/ResourceManagerTest.java   | 108 +++++++++++++++++
 .../flink/runtime/jobmaster/JobMasterTest.java  |  38 +-----
 .../resourcemanager/ResourceManagerHATest.java  |   7 ++
 .../ResourceManagerJobMasterTest.java           |   7 ++
 .../ResourceManagerTaskExecutorTest.java        |   7 ++
 .../slotmanager/SlotProtocolTest.java           |  17 +++
 .../taskexecutor/TaskExecutorITCase.java        |   3 +
 .../runtime/taskexecutor/TaskExecutorTest.java  | 121 +++++++++++++++++--
 .../yarn/YarnFlinkApplicationMasterRunner.java  |  21 ++--
 .../apache/flink/yarn/YarnResourceManager.java  |   8 +-
 22 files changed, 640 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
new file mode 100644
index 0000000..e628db5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/TestingHeartbeatServices.java
@@ -0,0 +1,52 @@
+/*
+ * 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.runtime.heartbeat;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+
+public class TestingHeartbeatServices extends HeartbeatServices {
+
+	private final ScheduledExecutor scheduledExecutorToUse;
+
+	public TestingHeartbeatServices(long heartbeatInterval, long heartbeatTimeout, ScheduledExecutor scheduledExecutorToUse) {
+		super(heartbeatInterval, heartbeatTimeout);
+
+		this.scheduledExecutorToUse = Preconditions.checkNotNull(scheduledExecutorToUse);
+	}
+
+	@Override
+	public <I, O> HeartbeatManager<I, O> createHeartbeatManagerSender(
+		ResourceID resourceId,
+		HeartbeatListener<I, O> heartbeatListener,
+		ScheduledExecutor scheduledExecutor,
+		Logger log) {
+
+		return new HeartbeatManagerSenderImpl<>(
+			heartbeatInterval,
+			heartbeatTimeout,
+			resourceId,
+			heartbeatListener,
+			org.apache.flink.runtime.concurrent.Executors.directExecutor(),
+			scheduledExecutorToUse,
+			log);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index 243b57f..81fc541 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -1043,11 +1043,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 		@Override
 		public void notifyHeartbeatTimeout(ResourceID resourceID) {
-			log.info("Task manager with id {} timed out.", resourceID);
+			log.info("Task manager with id {} heartbeat timed out.", resourceID);
 
 			getSelf().disconnectTaskManager(
 				resourceID,
-				new TimeoutException("The heartbeat of TaskManager with id " + resourceID + " timed out."));
+				new TimeoutException("Task manager with id " + resourceID + " heartbeat timed out."));
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
index 25c4aba..2cfba7b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
@@ -241,7 +241,7 @@ public class MiniCluster {
 				// bring up the ResourceManager(s)
 				LOG.info("Starting {} ResourceManger(s)", numResourceManagers);
 				resourceManagerRunners = startResourceManagers(
-						configuration, haServices, metricRegistry, numResourceManagers, resourceManagerRpcServices);
+						configuration, haServices, heartbeatServices, metricRegistry, numResourceManagers, resourceManagerRpcServices);
 
 				// bring up the TaskManager(s) for the mini cluster
 				LOG.info("Starting {} TaskManger(s)", numTaskManagers);
@@ -508,6 +508,7 @@ public class MiniCluster {
 	protected ResourceManagerRunner[] startResourceManagers(
 			Configuration configuration,
 			HighAvailabilityServices haServices,
+			HeartbeatServices heartbeatServices,
 			MetricRegistry metricRegistry,
 			int numResourceManagers,
 			RpcService[] resourceManagerRpcServices) throws Exception {
@@ -517,9 +518,11 @@ public class MiniCluster {
 		for (int i = 0; i < numResourceManagers; i++) {
 
 			resourceManagerRunners[i] = new ResourceManagerRunner(
+				ResourceID.generate(),
 				configuration,
 				resourceManagerRpcServices[i],
 				haServices,
+				heartbeatServices,
 				metricRegistry);
 
 			resourceManagerRunners[i].start();

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationConnectionListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationConnectionListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationConnectionListener.java
new file mode 100644
index 0000000..360f982
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationConnectionListener.java
@@ -0,0 +1,40 @@
+/*
+ * 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.runtime.registration;
+
+/**
+ * Classes which want to be notified about the registration result by the {@link RegisteredRpcConnection}
+ * have to implement this interface.
+ */
+public interface RegistrationConnectionListener<Success extends RegistrationResponse.Success> {
+
+	/**
+	 * This method is called by the {@link RegisteredRpcConnection} when the registration is success.
+	 *
+	 * @param success The concrete response information for successful registration.
+	 */
+	void onRegistrationSuccess(Success success);
+
+	/**
+	 * This method is called by the {@link RegisteredRpcConnection} when the registration fails.
+	 *
+	 * @param failure The exception which causes the registration failure.
+	 */
+	void onRegistrationFailure(Throwable failure);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
index 1430a49..9a7a790 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -30,6 +30,10 @@ import org.apache.flink.runtime.concurrent.ApplyFunction;
 import org.apache.flink.runtime.concurrent.BiFunction;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.heartbeat.HeartbeatListener;
+import org.apache.flink.runtime.heartbeat.HeartbeatManager;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.heartbeat.HeartbeatTarget;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.LeaderIdMismatchException;
 import org.apache.flink.runtime.instance.InstanceID;
@@ -64,6 +68,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executor;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -81,6 +86,9 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		extends RpcEndpoint<ResourceManagerGateway>
 		implements LeaderContender {
 
+	/** Unique id of the resource manager */
+	private final ResourceID resourceId;
+
 	/** Configuration of the resource manager */
 	private final ResourceManagerConfiguration resourceManagerConfiguration;
 
@@ -96,6 +104,9 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	/** High availability services for leader retrieval and election. */
 	private final HighAvailabilityServices highAvailabilityServices;
 
+	/** The heartbeat manager with task managers. */
+	private final HeartbeatManager<Void, Void> taskManagerHeartbeatManager;
+
 	/** The factory to construct the SlotManager. */
 	private final SlotManagerFactory slotManagerFactory;
 
@@ -118,9 +129,11 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	private ConcurrentMap<String, InfoMessageListenerRpcGateway> infoMessageListeners;
 
 	public ResourceManager(
+			ResourceID resourceId,
 			RpcService rpcService,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
 			SlotManagerFactory slotManagerFactory,
 			MetricRegistry metricRegistry,
 			JobLeaderIdService jobLeaderIdService,
@@ -128,6 +141,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 
 		super(rpcService);
 
+		this.resourceId = checkNotNull(resourceId);
 		this.resourceManagerConfiguration = checkNotNull(resourceManagerConfiguration);
 		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.slotManagerFactory = checkNotNull(slotManagerFactory);
@@ -135,6 +149,12 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		this.jobLeaderIdService = checkNotNull(jobLeaderIdService);
 		this.fatalErrorHandler = checkNotNull(fatalErrorHandler);
 
+		this.taskManagerHeartbeatManager = heartbeatServices.createHeartbeatManagerSender(
+				resourceId,
+				new TaskManagerHeartbeatListener(),
+				rpcService.getScheduledExecutor(),
+				log);
+
 		this.jobManagerRegistrations = new HashMap<>(4);
 		this.taskExecutors = new HashMap<>(8);
 		this.leaderSessionId = null;
@@ -178,6 +198,8 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	public void shutDown() throws Exception {
 		Exception exception = null;
 
+		taskManagerHeartbeatManager.stop();
+
 		try {
 			super.shutDown();
 		} catch (Exception e) {
@@ -326,7 +348,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	 *
 	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader
 	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 * @param taskExecutorResourceId  The resource ID of the TaskExecutor that registers
 	 *
 	 * @return The response by the ResourceManager.
 	 */
@@ -334,7 +356,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	public Future<RegistrationResponse> registerTaskExecutor(
 		final UUID resourceManagerLeaderId,
 		final String taskExecutorAddress,
-		final ResourceID resourceID,
+		final ResourceID taskExecutorResourceId,
 		final SlotReport slotReport) {
 
 		if (leaderSessionId.equals(resourceManagerLeaderId)) {
@@ -342,25 +364,37 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 
 			return taskExecutorGatewayFuture.handleAsync(new BiFunction<TaskExecutorGateway, Throwable, RegistrationResponse>() {
 				@Override
-				public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
+				public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway, Throwable throwable) {
 					if (throwable != null) {
 						return new RegistrationResponse.Decline(throwable.getMessage());
 					} else {
-						WorkerRegistration<WorkerType> oldRegistration = taskExecutors.remove(resourceID);
+						WorkerRegistration<WorkerType> oldRegistration = taskExecutors.remove(taskExecutorResourceId);
 						if (oldRegistration != null) {
 							// TODO :: suggest old taskExecutor to stop itself
-							log.info("Replacing old instance of worker for ResourceID {}", resourceID);
+							log.info("Replacing old instance of worker for ResourceID {}", taskExecutorResourceId);
 						}
 
-						WorkerType newWorker = workerStarted(resourceID);
+						WorkerType newWorker = workerStarted(taskExecutorResourceId);
 						WorkerRegistration<WorkerType> registration =
 							new WorkerRegistration<>(taskExecutorGateway, newWorker);
 
-						taskExecutors.put(resourceID, registration);
-						slotManager.registerTaskExecutor(resourceID, registration, slotReport);
+						taskExecutors.put(taskExecutorResourceId, registration);
+						slotManager.registerTaskExecutor(taskExecutorResourceId, registration, slotReport);
+
+						taskManagerHeartbeatManager.monitorTarget(taskExecutorResourceId, new HeartbeatTarget<Void>() {
+							@Override
+							public void receiveHeartbeat(ResourceID resourceID, Void payload) {
+								// the task manager will not request heartbeat, so this method will never be called currently
+							}
+
+							@Override
+							public void requestHeartbeat(ResourceID resourceID, Void payload) {
+								taskExecutorGateway.heartbeatFromResourceManager(resourceID);
+							}
+						});
 
 						return new TaskExecutorRegistrationSuccess(
-							registration.getInstanceID(),
+							registration.getInstanceID(), resourceId,
 							resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds());
 					}
 				}
@@ -368,7 +402,7 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		} else {
 			log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " +
 					"not equal the received leader session ID  {}",
-				resourceID, taskExecutorAddress, leaderSessionId, resourceManagerLeaderId);
+				taskExecutorResourceId, taskExecutorAddress, leaderSessionId, resourceManagerLeaderId);
 
 			return FlinkCompletableFuture.<RegistrationResponse>completed(
 				new RegistrationResponse.Decline("Discard registration because the leader id " +
@@ -377,6 +411,16 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 		}
 	}
 
+	@RpcMethod
+	public void heartbeatFromTaskManager(final ResourceID resourceID) {
+		taskManagerHeartbeatManager.receiveHeartbeat(resourceID, null);
+	}
+
+	@RpcMethod
+	public void disconnectTaskManager(final ResourceID resourceId, final Exception cause) {
+		closeTaskManagerConnection(resourceId, cause);
+	}
+
 	/**
 	 * Requests a slot from the resource manager.
 	 *
@@ -716,24 +760,24 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 	 * This method should be called by the framework once it detects that a currently registered
 	 * task executor has failed.
 	 *
-	 * @param resourceID Id of the worker that has failed.
-	 * @param message An informational message that explains why the worker failed.
+	 * @param resourceID Id of the TaskManager that has failed.
+	 * @param cause The exception which cause the TaskManager failed.
 	 */
-	public void notifyWorkerFailed(final ResourceID resourceID, final String message) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				WorkerRegistration<WorkerType> workerRegistration = taskExecutors.remove(resourceID);
+	public void closeTaskManagerConnection(final ResourceID resourceID, final Exception cause) {
+		taskManagerHeartbeatManager.unmonitorTarget(resourceID);
 
-				if (workerRegistration != null) {
-					log.info("Task manager {} failed because {}.", resourceID, message);
-					// TODO :: suggest failed task executor to stop itself
-					slotManager.notifyTaskManagerFailure(resourceID);
-				} else {
-					log.debug("Could not find a registered task manager with the process id {}.", resourceID);
-				}
-			}
-		});
+		WorkerRegistration<WorkerType> workerRegistration = taskExecutors.remove(resourceID);
+
+		if (workerRegistration != null) {
+			log.info("Task manager {} failed because {}.", resourceID, cause);
+
+			// TODO :: suggest failed task executor to stop itself
+			slotManager.notifyTaskManagerFailure(resourceID);
+
+			workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause);
+		} else {
+			log.debug("Could not find a registered task manager with the process id {}.", resourceID);
+		}
 	}
 
 	// ------------------------------------------------------------------------
@@ -827,5 +871,26 @@ public abstract class ResourceManager<WorkerType extends Serializable>
 			onFatalErrorAsync(error);
 		}
 	}
+
+	private class TaskManagerHeartbeatListener implements HeartbeatListener<Void, Void> {
+
+		@Override
+		public void notifyHeartbeatTimeout(ResourceID resourceID) {
+			log.info("The heartbeat of TaskManager with id {} timed out.", resourceID);
+
+			closeTaskManagerConnection(resourceID, new TimeoutException(
+					"Task manager with id " + resourceID + " heartbeat timed out."));
+		}
+
+		@Override
+		public void reportPayload(ResourceID resourceID, Void payload) {
+			// nothing to do since there is no payload
+		}
+
+		@Override
+		public Future<Void> retrievePayload() {
+			return FlinkCompletableFuture.completed(null);
+		}
+	}
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
index 8235ea7..7741e0d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -130,4 +130,20 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @return The future to the number of registered TaskManagers.
 	 */
 	Future<Integer> getNumberOfRegisteredTaskManagers(UUID leaderSessionId);
+
+	/**
+	 * Sends the heartbeat to resource manager from task manager
+	 *
+	 * @param resourceID unique id of the task manager
+	 */
+	void heartbeatFromTaskManager(final ResourceID resourceID);
+
+	/**
+	 * Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
+	 * {@link ResourceManager}.
+	 *
+	 * @param resourceID identifying the TaskManager to disconnect
+	 * @param cause for the disconnection of the TaskManager
+	 */
+	void disconnectTaskManager(ResourceID resourceID, Exception cause);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
index 73b27b5..d07e373 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.rpc.FatalErrorHandler;
@@ -43,14 +45,18 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 	private final ResourceManager<?> resourceManager;
 
 	public ResourceManagerRunner(
+			final ResourceID resourceId,
 			final Configuration configuration,
 			final RpcService rpcService,
 			final HighAvailabilityServices highAvailabilityServices,
+			final HeartbeatServices heartbeatServices,
 			final MetricRegistry metricRegistry) throws Exception {
 
+		Preconditions.checkNotNull(resourceId);
 		Preconditions.checkNotNull(configuration);
 		Preconditions.checkNotNull(rpcService);
 		Preconditions.checkNotNull(highAvailabilityServices);
+		Preconditions.checkNotNull(heartbeatServices);
 		Preconditions.checkNotNull(metricRegistry);
 
 		final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration);
@@ -63,9 +69,11 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 			rpcService.getScheduledExecutor());
 
 		this.resourceManager = new StandaloneResourceManager(
+			resourceId,
 			rpcService,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
+			heartbeatServices,
 			resourceManagerRuntimeServices.getSlotManagerFactory(),
 			metricRegistry,
 			resourceManagerRuntimeServices.getJobLeaderIdService(),
@@ -87,7 +95,6 @@ public class ResourceManagerRunner implements FatalErrorHandler {
 	private void shutDownInternally() throws Exception {
 		Exception exception = null;
 		synchronized (lock) {
-
 			try {
 				resourceManager.shutDown();
 			} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
index 73c8a2d..e2d6538 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 import org.apache.flink.runtime.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException;
@@ -37,17 +38,21 @@ import org.apache.flink.runtime.rpc.RpcService;
 public class StandaloneResourceManager extends ResourceManager<ResourceID> {
 
 	public StandaloneResourceManager(
+			ResourceID resourceId,
 			RpcService rpcService,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
 			SlotManagerFactory slotManagerFactory,
 			MetricRegistry metricRegistry,
 			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 		super(
+			resourceId,
 			rpcService,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
+			heartbeatServices,
 			slotManagerFactory,
 			metricRegistry,
 			jobLeaderIdService,

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index 83c225f..f3e1ff3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -57,6 +57,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.registration.RegistrationConnectionListener;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
@@ -135,6 +136,9 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/** The heartbeat manager for job manager in the task manager */
 	private final HeartbeatManager<Void, Void> jobManagerHeartbeatManager;
 
+	/** The heartbeat manager for resource manager in the task manager */
+	private final HeartbeatManager<Void, Void> resourceManagerHeartbeatManager;
+
 	/** The fatal error handler to use in case of a fatal error */
 	private final FatalErrorHandler fatalErrorHandler;
 
@@ -206,6 +210,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			new JobManagerHeartbeatListener(),
 			rpcService.getScheduledExecutor(),
 			log);
+
+		this.resourceManagerHeartbeatManager = heartbeatServices.createHeartbeatManager(
+				getResourceID(),
+				new ResourceManagerHeartbeatListener(),
+				rpcService.getScheduledExecutor(),
+				log);
 	}
 
 	// ------------------------------------------------------------------------
@@ -247,6 +257,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 		jobManagerHeartbeatManager.stop();
 
+		resourceManagerHeartbeatManager.stop();
+
 		ioManager.shutdown();
 
 		memoryManager.shutdown();
@@ -497,6 +509,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		jobManagerHeartbeatManager.requestHeartbeat(resourceID, null);
 	}
 
+	@RpcMethod
+	public void heartbeatFromResourceManager(ResourceID resourceID) {
+		resourceManagerHeartbeatManager.requestHeartbeat(resourceID, null);
+	}
+
 	// ----------------------------------------------------------------------
 	// Checkpointing RPCs
 	// ----------------------------------------------------------------------
@@ -619,11 +636,20 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		return new TMSlotRequestRegistered(resourceManagerConnection.getRegistrationId(), getResourceID(), allocationId);
 	}
 
+	// ----------------------------------------------------------------------
+	// Disconnection RPCs
+	// ----------------------------------------------------------------------
+
 	@RpcMethod
 	public void disconnectJobManager(JobID jobId, Exception cause) {
 		closeJobManagerConnection(jobId, cause);
 	}
 
+	@RpcMethod
+	public void disconnectResourceManager(Exception cause) {
+		closeResourceManagerConnection(cause);
+	}
+
 	// ======================================================================
 	//  Internal methods
 	// ======================================================================
@@ -665,11 +691,25 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 					newLeaderAddress,
 					newLeaderId,
 					getMainThreadExecutor(),
-					new ForwardingFatalErrorHandler());
+					new ResourceManagerRegistrationListener());
 			resourceManagerConnection.start();
 		}
 	}
 
+	private void closeResourceManagerConnection(Exception cause) {
+		log.info("Close ResourceManager connection for {}.", cause);
+
+		if (isConnectedToResourceManager()) {
+			resourceManagerHeartbeatManager.unmonitorTarget(resourceManagerConnection.getResourceManagerId());
+
+			ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
+			resourceManagerConnection.close();
+			resourceManagerConnection = null;
+
+			resourceManagerGateway.disconnectTaskManager(getResourceID(), cause);
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  Internal job manager connection methods
 	// ------------------------------------------------------------------------
@@ -747,10 +787,10 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 							offerSlotsToJobManager(jobId);
 						} else {
 							log.warn("Slot offering to JobManager failed. Freeing the slots " +
-								"and returning them to the ResourceManager.", throwable);
+									"and returning them to the ResourceManager.", throwable);
 
 							// We encountered an exception. Free the slots and return them to the RM.
-							for (SlotOffer reservedSlot: reservedSlots) {
+							for (SlotOffer reservedSlot : reservedSlots) {
 								freeSlot(reservedSlot.getAllocationId(), throwable);
 							}
 						}
@@ -1137,11 +1177,32 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-	private final class ForwardingFatalErrorHandler implements FatalErrorHandler {
+	private final class ResourceManagerRegistrationListener implements RegistrationConnectionListener<TaskExecutorRegistrationSuccess> {
 
 		@Override
-		public void onFatalError(Throwable exception) {
-			onFatalErrorAsync(exception);
+		public void onRegistrationSuccess(TaskExecutorRegistrationSuccess success) {
+			final ResourceID resourceManagerId = success.getResourceManagerId();
+
+			// monitor the resource manager as heartbeat target
+			resourceManagerHeartbeatManager.monitorTarget(resourceManagerId, new HeartbeatTarget<Void>() {
+				@Override
+				public void receiveHeartbeat(ResourceID resourceID, Void payload) {
+					if (isConnectedToResourceManager()) {
+						ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway();
+						resourceManagerGateway.heartbeatFromTaskManager(resourceID);
+					}
+				}
+
+				@Override
+				public void requestHeartbeat(ResourceID resourceID, Void payload) {
+					// request heartbeat will never be called on the task manager side
+				}
+			});
+		}
+
+		@Override
+		public void onRegistrationFailure(Throwable failure) {
+			onFatalErrorAsync(failure);
 		}
 	}
 
@@ -1216,15 +1277,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			runAsync(new Runnable() {
 				@Override
 				public void run() {
-					log.info("The JobManager connection {} has timed out.", resourceID);
+					log.info("Job manager with id {} heartbeat timed out.", resourceID);
 
 					if (jobManagerConnections.containsKey(resourceID)) {
 						JobManagerConnection jobManagerConnection = jobManagerConnections.get(resourceID);
 						if (jobManagerConnection != null) {
 							closeJobManagerConnection(
 								jobManagerConnection.getJobID(),
-								new TimeoutException("The heartbeat of JobManager with id " +
-									resourceID + " timed out."));
+								new TimeoutException("Job manager with id " + resourceID + " heartbeat timed out."));
 						}
 					}
 				}
@@ -1241,4 +1301,32 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			return FlinkCompletableFuture.completed(null);
 		}
 	}
+
+	private class ResourceManagerHeartbeatListener implements HeartbeatListener<Void, Void> {
+
+		@Override
+		public void notifyHeartbeatTimeout(final ResourceID resourceID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("Resource manager with id {} heartbeat timed out.", resourceID);
+
+					if (isConnectedToResourceManager() && resourceManagerConnection.getResourceManagerId().equals(resourceID)) {
+						closeResourceManagerConnection(
+								new TimeoutException("Resource manager with id " + resourceID + " heartbeat timed out."));
+					}
+				}
+			});
+		}
+
+		@Override
+		public void reportPayload(ResourceID resourceID, Void payload) {
+			// nothing to do since the payload is of type Void
+		}
+
+		@Override
+		public Future<Void> retrievePayload() {
+			return FlinkCompletableFuture.completed(null);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
index 2dcc3a4..2bbf0e6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
@@ -141,10 +141,24 @@ public interface TaskExecutorGateway extends RpcGateway {
 	void heartbeatFromJobManager(ResourceID heartbeatOrigin);
 
 	/**
+	 * Heartbeat request from the resource manager
+	 *
+	 * @param heartbeatOrigin unique id of the resource manager
+	 */
+	void heartbeatFromResourceManager(ResourceID heartbeatOrigin);
+
+	/**
 	 * Disconnects the given JobManager from the TaskManager.
 	 *
 	 * @param jobId JobID for which the JobManager was the leader
 	 * @param cause for the disconnection from the JobManager
 	 */
 	void disconnectJobManager(JobID jobId, Exception cause);
+
+	/**
+	 * Disconnects the ResourceManager from the TaskManager.
+	 *
+	 * @param cause for the disconnection from the ResourceManager
+	 */
+	void disconnectResourceManager(Exception cause);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
index b357f52..4b61f68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.taskexecutor;
 
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 
@@ -33,16 +34,20 @@ public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.
 
 	private final InstanceID registrationId;
 
+	private final ResourceID resourceManagerResourceId;
+
 	private final long heartbeatInterval;
 
 	/**
 	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
 	 * 
 	 * @param registrationId     The ID that the ResourceManager assigned the registration.
+	 * @param resourceManagerResourceId The unique ID that identifies the ResourceManager.
 	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
 	 */
-	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
+	public TaskExecutorRegistrationSuccess(InstanceID registrationId, ResourceID resourceManagerResourceId, long heartbeatInterval) {
 		this.registrationId = registrationId;
+		this.resourceManagerResourceId = resourceManagerResourceId;
 		this.heartbeatInterval = heartbeatInterval;
 	}
 
@@ -54,6 +59,13 @@ public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.
 	}
 
 	/**
+	 * Gets the unique ID that identifies the ResourceManager.
+	 */
+	public ResourceID getResourceManagerId() {
+		return resourceManagerResourceId;
+	}
+
+	/**
 	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
 	 */
 	public long getHeartbeatInterval() {
@@ -62,7 +74,7 @@ public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.
 
 	@Override
 	public String toString() {
-		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
+		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + resourceManagerResourceId + " / " + heartbeatInterval + ')';
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
index 6e3e39b..775482c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.registration.RegisteredRpcConnection;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.registration.RegistrationConnectionListener;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.registration.RetryingRegistration;
@@ -51,10 +51,12 @@ public class TaskExecutorToResourceManagerConnection
 
 	private final SlotReport slotReport;
 
-	private final FatalErrorHandler fatalErrorHandler;
+	private final RegistrationConnectionListener<TaskExecutorRegistrationSuccess> registrationListener;
 
 	private InstanceID registrationId;
 
+	private ResourceID resourceManagerResourceId;
+
 	public TaskExecutorToResourceManagerConnection(
 			Logger log,
 			RpcService rpcService,
@@ -64,7 +66,7 @@ public class TaskExecutorToResourceManagerConnection
 			String resourceManagerAddress,
 			UUID resourceManagerLeaderId,
 			Executor executor,
-			FatalErrorHandler fatalErrorHandler) {
+			RegistrationConnectionListener<TaskExecutorRegistrationSuccess> registrationListener) {
 
 		super(log, resourceManagerAddress, resourceManagerLeaderId, executor);
 
@@ -72,7 +74,7 @@ public class TaskExecutorToResourceManagerConnection
 		this.taskManagerAddress = Preconditions.checkNotNull(taskManagerAddress);
 		this.taskManagerResourceId = Preconditions.checkNotNull(taskManagerResourceId);
 		this.slotReport = Preconditions.checkNotNull(slotReport);
-		this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler);
+		this.registrationListener = Preconditions.checkNotNull(registrationListener);
 	}
 
 
@@ -94,13 +96,15 @@ public class TaskExecutorToResourceManagerConnection
 			getTargetAddress(), success.getRegistrationId());
 
 		registrationId = success.getRegistrationId();
+		resourceManagerResourceId = success.getResourceManagerId();
+		registrationListener.onRegistrationSuccess(success);
 	}
 
 	@Override
 	protected void onRegistrationFailure(Throwable failure) {
 		log.info("Failed to register at resource manager {}.", getTargetAddress(), failure);
 
-		fatalErrorHandler.onFatalError(failure);
+		registrationListener.onRegistrationFailure(failure);
 	}
 
 	/**
@@ -111,6 +115,13 @@ public class TaskExecutorToResourceManagerConnection
 		return registrationId;
 	}
 
+	/**
+	 * Gets the unique id of ResourceManager, that is returned when registration success.
+	 */
+	public ResourceID getResourceManagerId() {
+		return resourceManagerResourceId;
+	}
+
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
index ca8a07a..e7f2439 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.clusterframework;
 
 import akka.actor.ActorSystem;
 import akka.testkit.JavaTestKit;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
@@ -30,21 +31,47 @@ import org.apache.flink.runtime.clusterframework.messages.RemoveResource;
 import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved;
 import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration;
+import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager;
+import org.apache.flink.runtime.resourcemanager.TestingSlotManagerFactory;
+import org.apache.flink.runtime.rpc.TestingSerialRpcService;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.apache.flink.runtime.util.TestingFatalErrorHandler;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 import scala.Option;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 /**
  * General tests for the resource manager component.
@@ -335,4 +362,85 @@ public class ResourceManagerTest {
 		}};
 		}};
 	}
+
+	@Test
+	public void testHeartbeatTimeoutWithTaskExecutor() throws Exception {
+		final String taskManagerAddress = "tm";
+		final ResourceID taskManagerResourceID = new ResourceID(taskManagerAddress);
+		final ResourceID resourceManagerResourceID = ResourceID.generate();
+		final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class);
+
+		final TestingSerialRpcService rpcService = new TestingSerialRpcService();
+		rpcService.registerGateway(taskManagerAddress, taskExecutorGateway);
+
+		final ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(
+			Time.seconds(5L),
+			Time.seconds(5L));
+
+		final TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
+		final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
+
+		final long heartbeatInterval = 1L;
+		final long heartbeatTimeout = 5L;
+		final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class);
+		final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor);
+
+		final TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+		final MetricRegistry metricRegistry = mock(MetricRegistry.class);
+		final JobLeaderIdService jobLeaderIdService = mock(JobLeaderIdService.class);
+		final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
+		try {
+			final StandaloneResourceManager resourceManager = new StandaloneResourceManager(
+				resourceManagerResourceID,
+				rpcService,
+				resourceManagerConfiguration,
+				highAvailabilityServices,
+				heartbeatServices,
+				slotManagerFactory,
+				metricRegistry,
+				jobLeaderIdService,
+				testingFatalErrorHandler);
+
+			resourceManager.start();
+
+			final UUID rmLeaderSessionId = UUID.randomUUID();
+			rmLeaderElectionService.isLeader(rmLeaderSessionId);
+
+			final SlotReport slotReport = new SlotReport();
+			// test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time
+			Future<RegistrationResponse> successfulFuture =
+					resourceManager.registerTaskExecutor(rmLeaderSessionId, taskManagerAddress, taskManagerResourceID, slotReport);
+			RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS);
+			assertTrue(response instanceof TaskExecutorRegistrationSuccess);
+
+			ArgumentCaptor<Runnable> heartbeatRunnableCaptor = ArgumentCaptor.forClass(Runnable.class);
+			verify(scheduledExecutor, times(1)).scheduleAtFixedRate(
+				heartbeatRunnableCaptor.capture(),
+				eq(0L),
+				eq(heartbeatInterval),
+				eq(TimeUnit.MILLISECONDS));
+
+			Runnable heartbeatRunnable = heartbeatRunnableCaptor.getValue();
+
+			ArgumentCaptor<Runnable> timeoutRunnableCaptor = ArgumentCaptor.forClass(Runnable.class);
+			verify(scheduledExecutor).schedule(timeoutRunnableCaptor.capture(), eq(heartbeatTimeout), eq(TimeUnit.MILLISECONDS));
+
+			Runnable timeoutRunnable = timeoutRunnableCaptor.getValue();
+
+			// run the first heartbeat request
+			heartbeatRunnable.run();
+
+			verify(taskExecutorGateway, times(1)).heartbeatFromResourceManager(eq(resourceManagerResourceID));
+
+			// run the timeout runnable to simulate a heartbeat timeout
+			timeoutRunnable.run();
+
+			verify(taskExecutorGateway).disconnectResourceManager(any(TimeoutException.class));
+
+		} finally {
+			rpcService.stopService();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
index 43536b6..73da244 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
@@ -26,10 +26,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor;
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
 import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoader;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
-import org.apache.flink.runtime.heartbeat.HeartbeatListener;
-import org.apache.flink.runtime.heartbeat.HeartbeatManager;
-import org.apache.flink.runtime.heartbeat.HeartbeatManagerSenderImpl;
-import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.heartbeat.*;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.OnCompletionActions;
@@ -38,14 +35,12 @@ import org.apache.flink.runtime.rpc.TestingSerialRpcService;
 import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.TestingFatalErrorHandler;
-import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.ArgumentCaptor;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
-import org.slf4j.Logger;
 
 import java.net.InetAddress;
 import java.net.URL;
@@ -108,10 +103,9 @@ public class JobMasterTest extends TestLogger {
 				testingFatalErrorHandler,
 				new FlinkUserCodeClassLoader(new URL[0]));
 
-			// also start the heartbeat manager in job manager
 			jobMaster.start(jmLeaderId);
 
-			// register task manager will trigger monitoring heartbeat target, schedule heartbeat request in interval time
+			// register task manager will trigger monitor heartbeat target, schedule heartbeat request at interval time
 			jobMaster.registerTaskManager(taskManagerAddress, taskManagerLocation, jmLeaderId);
 
 			ArgumentCaptor<Runnable> heartbeatRunnableCaptor = ArgumentCaptor.forClass(Runnable.class);
@@ -145,32 +139,4 @@ public class JobMasterTest extends TestLogger {
 			rpc.stopService();
 		}
 	}
-
-	private static class TestingHeartbeatServices extends HeartbeatServices {
-
-		private final ScheduledExecutor scheduledExecutorToUse;
-
-		public TestingHeartbeatServices(long heartbeatInterval, long heartbeatTimeout, ScheduledExecutor scheduledExecutorToUse) {
-			super(heartbeatInterval, heartbeatTimeout);
-
-			this.scheduledExecutorToUse = Preconditions.checkNotNull(scheduledExecutorToUse);
-		}
-
-		@Override
-		public <I, O> HeartbeatManager<I, O> createHeartbeatManagerSender(
-			ResourceID resourceId,
-			HeartbeatListener<I, O> heartbeatListener,
-			ScheduledExecutor scheduledExecutor,
-			Logger log) {
-
-			return new HeartbeatManagerSenderImpl<>(
-				heartbeatInterval,
-				heartbeatTimeout,
-				resourceId,
-				heartbeatListener,
-				org.apache.flink.runtime.concurrent.Executors.directExecutor(),
-				scheduledExecutorToUse,
-				log);
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
index 1aa799b..39594df 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.metrics.MetricRegistry;
@@ -40,12 +42,15 @@ public class ResourceManagerHATest {
 
 	@Test
 	public void testGrantAndRevokeLeadership() throws Exception {
+		ResourceID rmResourceId = ResourceID.generate();
 		RpcService rpcService = new TestingSerialRpcService();
 
 		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
 
+		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(
 			Time.seconds(5L),
 			Time.seconds(5L));
@@ -63,9 +68,11 @@ public class ResourceManagerHATest {
 
 		final ResourceManager resourceManager =
 			new StandaloneResourceManager(
+				rmResourceId,
 				rpcService,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
+				heartbeatServices,
 				slotManagerFactory,
 				metricRegistry,
 				resourceManagerRuntimeServices.getJobLeaderIdService(),

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
index 9a68eca..0401f9e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java
@@ -20,7 +20,9 @@ package org.apache.flink.runtime.resourcemanager;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
@@ -196,10 +198,13 @@ public class ResourceManagerJobMasterTest {
 			JobID jobID,
 			TestingLeaderRetrievalService jobMasterLeaderRetrievalService,
 			FatalErrorHandler fatalErrorHandler) throws Exception {
+		ResourceID rmResourceId = ResourceID.generate();
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
 		highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService);
 		highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService);
 
+		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(
 			Time.seconds(5L),
 			Time.seconds(5L));
@@ -211,9 +216,11 @@ public class ResourceManagerJobMasterTest {
 			Time.minutes(5L));
 
 		ResourceManager resourceManager = new StandaloneResourceManager(
+			rmResourceId,
 			rpcService,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
+			heartbeatServices,
 			slotManagerFactory,
 			metricRegistry,
 			jobLeaderIdService,

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
index 0a1addb..7c811d9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.resourcemanager;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.metrics.MetricRegistry;
@@ -52,6 +53,8 @@ public class ResourceManagerTaskExecutorTest {
 
 	private ResourceID taskExecutorResourceID;
 
+	private ResourceID resourceManagerResourceID;
+
 	private StandaloneResourceManager resourceManager;
 
 	private UUID leaderSessionId;
@@ -63,6 +66,7 @@ public class ResourceManagerTaskExecutorTest {
 		rpcService = new TestingSerialRpcService();
 
 		taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress);
+		resourceManagerResourceID = ResourceID.generate();
 		TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService();
 		testingFatalErrorHandler = new TestingFatalErrorHandler();
 		resourceManager = createAndStartResourceManager(rmLeaderElectionService, testingFatalErrorHandler);
@@ -144,6 +148,7 @@ public class ResourceManagerTaskExecutorTest {
 
 	private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception {
 		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
 		highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService);
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
 		ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration(
@@ -158,9 +163,11 @@ public class ResourceManagerTaskExecutorTest {
 
 		StandaloneResourceManager resourceManager =
 			new StandaloneResourceManager(
+				resourceManagerResourceID,
 				rpcService,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
+				heartbeatServices,
 				slotManagerFactory,
 				metricRegistry,
 				jobLeaderIdService,

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
index ea660f8..28ed697 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.clusterframework.types.SlotID;
 import org.apache.flink.runtime.concurrent.Future;
 import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
 import org.apache.flink.runtime.jobmaster.JobMasterGateway;
@@ -98,6 +99,7 @@ public class SlotProtocolTest extends TestLogger {
 		final String rmAddress = "/rm1";
 		final String jmAddress = "/jm1";
 		final JobID jobID = new JobID();
+		final ResourceID rmResourceId = new ResourceID(rmAddress);
 
 		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
 
@@ -117,11 +119,16 @@ public class SlotProtocolTest extends TestLogger {
 			Time.seconds(5L));
 
 		final TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+
+		final HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+
 		SpiedResourceManager resourceManager =
 			new SpiedResourceManager(
+				rmResourceId,
 				testRpcService,
 				resourceManagerConfiguration,
 				testingHaServices,
+				heartbeatServices,
 				slotManagerFactory,
 				mock(MetricRegistry.class),
 				jobLeaderIdService,
@@ -198,6 +205,7 @@ public class SlotProtocolTest extends TestLogger {
 		final String jmAddress = "/jm1";
 		final String tmAddress = "/tm1";
 		final JobID jobID = new JobID();
+		final ResourceID rmResourceId = new ResourceID(rmAddress);
 
 		testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class));
 
@@ -224,11 +232,16 @@ public class SlotProtocolTest extends TestLogger {
 			Time.seconds(5L));
 
 		TestingSlotManagerFactory slotManagerFactory = new TestingSlotManagerFactory();
+
+		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+
 		ResourceManager<ResourceID> resourceManager =
 			Mockito.spy(new StandaloneResourceManager(
+				rmResourceId,
 				testRpcService,
 				resourceManagerConfiguration,
 				testingHaServices,
+				heartbeatServices,
 				slotManagerFactory,
 				mock(MetricRegistry.class),
 				jobLeaderIdService,
@@ -302,17 +315,21 @@ public class SlotProtocolTest extends TestLogger {
 		private int startNewWorkerCalled = 0;
 
 		public SpiedResourceManager(
+				ResourceID resourceId,
 				RpcService rpcService,
 				ResourceManagerConfiguration resourceManagerConfiguration,
 				HighAvailabilityServices highAvailabilityServices,
+				HeartbeatServices heartbeatServices,
 				SlotManagerFactory slotManagerFactory,
 				MetricRegistry metricRegistry,
 				JobLeaderIdService jobLeaderIdService,
 				FatalErrorHandler fatalErrorHandler) {
 			super(
+				resourceId,
 				rpcService,
 				resourceManagerConfiguration,
 				highAvailabilityServices,
+				heartbeatServices,
 				slotManagerFactory,
 				metricRegistry,
 				jobLeaderIdService,

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
index f6c2dce..4e76486 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
@@ -87,6 +87,7 @@ public class TaskExecutorITCase {
 		final String rmAddress = "rm";
 		final String jmAddress = "jm";
 		final UUID jmLeaderId = UUID.randomUUID();
+		final ResourceID rmResourceId = new ResourceID(rmAddress);
 		final JobID jobId = new JobID();
 		final ResourceProfile resourceProfile = new ResourceProfile(1.0, 1);
 
@@ -119,9 +120,11 @@ public class TaskExecutorITCase {
 		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
 
 		ResourceManager<ResourceID> resourceManager = new StandaloneResourceManager(
+			rmResourceId,
 			rpcService,
 			resourceManagerConfiguration,
 			testingHAServices,
+			heartbeatServices,
 			slotManagerFactory,
 			metricRegistry,
 			jobLeaderIdService,

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
index 67196aa..d1f6e2e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -213,9 +213,104 @@ public class TaskExecutorTest extends TestLogger {
 	}
 
 	@Test
+	public void testHeartbeatTimeoutWithResourceManager() throws Exception {
+		final String rmAddress = "rm";
+		final String tmAddress = "tm";
+		final ResourceID rmResourceId = new ResourceID(rmAddress);
+		final ResourceID tmResourceId = new ResourceID(tmAddress);
+		final UUID rmLeaderId = UUID.randomUUID();
+
+		// register the mock resource manager gateway
+		ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+		when(rmGateway.registerTaskExecutor(
+			any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
+			.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(
+					new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId, 10L)));
+
+		final TestingSerialRpcService rpc = new TestingSerialRpcService();
+		rpc.registerGateway(rmAddress, rmGateway);
+
+		final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+		final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+		haServices.setResourceManagerLeaderRetriever(testLeaderService);
+
+		final TaskManagerConfiguration taskManagerConfiguration = mock(TaskManagerConfiguration.class);
+		when(taskManagerConfiguration.getNumberSlots()).thenReturn(1);
+
+		final TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
+		when(taskManagerLocation.getResourceID()).thenReturn(tmResourceId);
+
+		final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class);
+		final SlotReport slotReport = new SlotReport();
+		when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport);
+
+		final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler();
+
+		final long heartbeatTimeout = 10L;
+		HeartbeatServices heartbeatServices = mock(HeartbeatServices.class);
+		when(heartbeatServices.createHeartbeatManager(
+			eq(taskManagerLocation.getResourceID()),
+			any(HeartbeatListener.class),
+			any(ScheduledExecutor.class),
+			any(Logger.class))).thenAnswer(
+			new Answer<HeartbeatManagerImpl<Void, Void>>() {
+				@Override
+				public HeartbeatManagerImpl<Void, Void> answer(InvocationOnMock invocation) throws Throwable {
+					return new HeartbeatManagerImpl<>(
+						heartbeatTimeout,
+						taskManagerLocation.getResourceID(),
+						(HeartbeatListener<Void, Void>)invocation.getArguments()[1],
+						(Executor)invocation.getArguments()[2],
+						(ScheduledExecutor)invocation.getArguments()[2],
+						(Logger)invocation.getArguments()[3]);
+					}
+				}
+		);
+
+		try {
+			final TaskExecutor taskManager = new TaskExecutor(
+				taskManagerConfiguration,
+				taskManagerLocation,
+				rpc,
+				mock(MemoryManager.class),
+				mock(IOManager.class),
+				mock(NetworkEnvironment.class),
+				haServices,
+				heartbeatServices,
+				mock(MetricRegistry.class),
+				mock(TaskManagerMetricGroup.class),
+				mock(BroadcastVariableManager.class),
+				mock(FileCache.class),
+				taskSlotTable,
+				mock(JobManagerTable.class),
+				mock(JobLeaderService.class),
+				testingFatalErrorHandler);
+
+			taskManager.start();
+
+			// define a leader and see that a registration happens
+			testLeaderService.notifyListener(rmAddress, rmLeaderId);
+
+			// register resource manager success will trigger monitoring heartbeat target between tm and rm
+			verify(rmGateway).registerTaskExecutor(
+					eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), any(SlotReport.class), any(Time.class));
+
+			// heartbeat timeout should trigger disconnect TaskManager from ResourceManager
+			verify(rmGateway, timeout(heartbeatTimeout * 5)).disconnectTaskManager(eq(taskManagerLocation.getResourceID()), any(TimeoutException.class));
+
+			// check if a concurrent error occurred
+			testingFatalErrorHandler.rethrowError();
+
+		} finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
 	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
 		final ResourceID resourceID = ResourceID.generate();
 		final String resourceManagerAddress = "/resource/manager/address/one";
+		final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress);
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
@@ -223,7 +318,8 @@ public class TaskExecutorTest extends TestLogger {
 			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
 			when(rmGateway.registerTaskExecutor(
 					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
-				.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(new InstanceID(), 10L)));
+				.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(
+					new InstanceID(), resourceManagerResourceId, 10L)));
 
 			TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class);
 			when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1);
@@ -275,12 +371,14 @@ public class TaskExecutorTest extends TestLogger {
 
 	@Test
 	public void testTriggerRegistrationOnLeaderChange() throws Exception {
-		final ResourceID resourceID = ResourceID.generate();
+		final ResourceID tmResourceID = ResourceID.generate();
 
 		final String address1 = "/resource/manager/address/one";
 		final String address2 = "/resource/manager/address/two";
 		final UUID leaderId1 = UUID.randomUUID();
 		final UUID leaderId2 = UUID.randomUUID();
+		final ResourceID rmResourceId1 = new ResourceID(address1);
+		final ResourceID rmResourceId2 = new ResourceID(address2);
 
 		final TestingSerialRpcService rpc = new TestingSerialRpcService();
 		try {
@@ -291,11 +389,11 @@ public class TaskExecutorTest extends TestLogger {
 			when(rmGateway1.registerTaskExecutor(
 					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
 					.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(
-						new TaskExecutorRegistrationSuccess(new InstanceID(), 10L)));
+						new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L)));
 			when(rmGateway2.registerTaskExecutor(
 					any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class)))
 					.thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(
-						new TaskExecutorRegistrationSuccess(new InstanceID(), 10L)));
+						new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L)));
 
 			rpc.registerGateway(address1, rmGateway1);
 			rpc.registerGateway(address2, rmGateway2);
@@ -313,7 +411,7 @@ public class TaskExecutorTest extends TestLogger {
 			when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]);
 
 			TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class);
-			when(taskManagerLocation.getResourceID()).thenReturn(resourceID);
+			when(taskManagerLocation.getResourceID()).thenReturn(tmResourceID);
 			when(taskManagerLocation.getHostname()).thenReturn("foobar");
 
 			final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class);
@@ -350,7 +448,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address1, leaderId1);
 
 			verify(rmGateway1).registerTaskExecutor(
-					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class));
+					eq(leaderId1), eq(taskManagerAddress), eq(tmResourceID), any(SlotReport.class), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 
 			// cancel the leader 
@@ -360,7 +458,7 @@ public class TaskExecutorTest extends TestLogger {
 			testLeaderService.notifyListener(address2, leaderId2);
 
 			verify(rmGateway2).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class));
+					eq(leaderId2), eq(taskManagerAddress), eq(tmResourceID), eq(slotReport), any(Time.class));
 			assertNotNull(taskManager.getResourceManagerConnection());
 
 			// check if a concurrent error occurred
@@ -531,6 +629,7 @@ public class TaskExecutorTest extends TestLogger {
 
 		final String resourceManagerAddress = "rm";
 		final UUID resourceManagerLeaderId = UUID.randomUUID();
+		final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress);
 
 		final ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class);
 		final InstanceID registrationId = new InstanceID();
@@ -540,7 +639,7 @@ public class TaskExecutorTest extends TestLogger {
 			any(String.class),
 			eq(resourceId),
 			any(SlotReport.class),
-			any(Time.class))).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, 1000L)));
+			any(Time.class))).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L)));
 
 		final String jobManagerAddress = "jm";
 		final UUID jobManagerLeaderId = UUID.randomUUID();
@@ -638,6 +737,7 @@ public class TaskExecutorTest extends TestLogger {
 
 		final String resourceManagerAddress = "rm";
 		final UUID resourceManagerLeaderId = UUID.randomUUID();
+		final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress);
 
 		final String jobManagerAddress = "jm";
 		final UUID jobManagerLeaderId = UUID.randomUUID();
@@ -655,7 +755,7 @@ public class TaskExecutorTest extends TestLogger {
 			any(String.class),
 			eq(resourceId),
 			any(SlotReport.class),
-			any(Time.class))).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, 1000L)));
+			any(Time.class))).thenReturn(FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L)));
 
 		final ResourceID jmResourceId = new ResourceID(jobManagerAddress);
 		final int blobPort = 42;
@@ -844,6 +944,7 @@ public class TaskExecutorTest extends TestLogger {
 
 		final String resourceManagerAddress = "rm";
 		final UUID resourceManagerLeaderId = UUID.randomUUID();
+		final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress);
 
 		final String jobManagerAddress = "jm";
 		final UUID jobManagerLeaderId = UUID.randomUUID();
@@ -862,7 +963,7 @@ public class TaskExecutorTest extends TestLogger {
 			eq(resourceId),
 			any(SlotReport.class),
 			any(Time.class))).thenReturn(
-				FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, 1000L)));
+				FlinkCompletableFuture.<RegistrationResponse>completed(new TaskExecutorRegistrationSuccess(registrationId, resourceManagerResourceId, 1000L)));
 
 		final ResourceID jmResourceId = new ResourceID(jobManagerAddress);
 		final int blobPort = 42;

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
index 7a0dbbe..ed672a3 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkApplicationMasterRunner.java
@@ -198,15 +198,18 @@ public class YarnFlinkApplicationMasterRunner extends AbstractYarnFlinkApplicati
 			haServices,
 			commonRpcService.getScheduledExecutor());
 
-		return new YarnResourceManager(config,
-				ENV,
-				commonRpcService,
-				resourceManagerConfiguration,
-				haServices,
-				resourceManagerRuntimeServices.getSlotManagerFactory(),
-				metricRegistry,
-				resourceManagerRuntimeServices.getJobLeaderIdService(),
-				this);
+		return new YarnResourceManager(
+			ResourceID.generate(),
+			config,
+			ENV,
+			commonRpcService,
+			resourceManagerConfiguration,
+			haServices,
+			heartbeatServices,
+			resourceManagerRuntimeServices.getSlotManagerFactory(),
+			metricRegistry,
+			resourceManagerRuntimeServices.getJobLeaderIdService(),
+			this);
 	}
 
 	private JobManagerRunner createJobManagerRunner(Configuration config) throws Exception{

http://git-wip-us.apache.org/repos/asf/flink/blob/fd90672f/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
index ab96441..a308079 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.resourcemanager.JobLeaderIdService;
@@ -106,19 +107,23 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	final private Map<ResourceProfile, Integer> resourcePriorities = new HashMap<>();
 
 	public YarnResourceManager(
+			ResourceID resourceId,
 			Configuration flinkConfig,
 			Map<String, String> env,
 			RpcService rpcService,
 			ResourceManagerConfiguration resourceManagerConfiguration,
 			HighAvailabilityServices highAvailabilityServices,
+			HeartbeatServices heartbeatServices,
 			SlotManagerFactory slotManagerFactory,
 			MetricRegistry metricRegistry,
 			JobLeaderIdService jobLeaderIdService,
 			FatalErrorHandler fatalErrorHandler) {
 		super(
+			resourceId,
 			rpcService,
 			resourceManagerConfiguration,
 			highAvailabilityServices,
+			heartbeatServices,
 			slotManagerFactory,
 			metricRegistry,
 			jobLeaderIdService,
@@ -231,7 +236,8 @@ public class YarnResourceManager extends ResourceManager<ResourceID> implements
 	public void onContainersCompleted(List<ContainerStatus> list) {
 		for (ContainerStatus container : list) {
 			if (container.getExitStatus() < 0) {
-				notifyWorkerFailed(new ResourceID(container.getContainerId().toString()), container.getDiagnostics());
+				closeTaskManagerConnection(new ResourceID(
+					container.getContainerId().toString()), new Exception(container.getDiagnostics()));
 			}
 		}
 	}


[36/50] [abbrv] flink git commit: [FLINK-6207] Duplicate TypeSerializers for async snapshots of CopyOnWriteStateTable

Posted by fh...@apache.org.
[FLINK-6207] Duplicate TypeSerializers for async snapshots of CopyOnWriteStateTable


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

Branch: refs/heads/table-retraction
Commit: 89866a5ad0673dd448b3cff186e6d663189a60e3
Parents: ad21a44
Author: Stefan Richter <s....@data-artisans.com>
Authored: Tue Mar 28 16:56:27 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Tue Mar 28 19:20:53 2017 +0200

----------------------------------------------------------------------
 .../heap/CopyOnWriteStateTableSnapshot.java     |  34 +++-
 .../state/heap/CopyOnWriteStateTableTest.java   | 171 ++++++++++++++++++-
 2 files changed, 195 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/89866a5a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java
index c83fce0..2ac88b3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java
@@ -71,6 +71,21 @@ public class CopyOnWriteStateTableSnapshot<K, N, S>
 	private int[] keyGroupOffsets;
 
 	/**
+	 * A local duplicate of the table's key serializer.
+	 */
+	private final TypeSerializer<K> localKeySerializer;
+
+	/**
+	 * A local duplicate of the table's namespace serializer.
+	 */
+	private final TypeSerializer<N> localNamespaceSerializer;
+
+	/**
+	 * A local duplicate of the table's state serializer.
+	 */
+	private final TypeSerializer<S> localStateSerializer;
+
+	/**
 	 * Creates a new {@link CopyOnWriteStateTableSnapshot}.
 	 *
 	 * @param owningStateTable the {@link CopyOnWriteStateTable} for which this object represents a snapshot.
@@ -81,6 +96,13 @@ public class CopyOnWriteStateTableSnapshot<K, N, S>
 		this.snapshotData = owningStateTable.snapshotTableArrays();
 		this.snapshotVersion = owningStateTable.getStateTableVersion();
 		this.stateTableSize = owningStateTable.size();
+
+		// We create duplicates of the serializers for the async snapshot, because TypeSerializer
+		// might be stateful and shared with the event processing thread.
+		this.localKeySerializer = owningStateTable.keyContext.getKeySerializer().duplicate();
+		this.localNamespaceSerializer = owningStateTable.metaInfo.getNamespaceSerializer().duplicate();
+		this.localStateSerializer = owningStateTable.metaInfo.getStateSerializer().duplicate();
+
 		this.keyGroupOffsets = null;
 	}
 
@@ -162,10 +184,6 @@ public class CopyOnWriteStateTableSnapshot<K, N, S>
 		int startOffset = keyGroupOffsetIdx < 0 ? 0 : keyGroupOffsets[keyGroupOffsetIdx];
 		int endOffset = keyGroupOffsets[keyGroupOffsetIdx + 1];
 
-		TypeSerializer<K> keySerializer = owningStateTable.keyContext.getKeySerializer();
-		TypeSerializer<N> namespaceSerializer = owningStateTable.metaInfo.getNamespaceSerializer();
-		TypeSerializer<S> stateSerializer = owningStateTable.metaInfo.getStateSerializer();
-
 		// write number of mappings in key-group
 		dov.writeInt(endOffset - startOffset);
 
@@ -173,9 +191,9 @@ public class CopyOnWriteStateTableSnapshot<K, N, S>
 		for (int i = startOffset; i < endOffset; ++i) {
 			CopyOnWriteStateTable.StateTableEntry<K, N, S> toWrite = groupedOut[i];
 			groupedOut[i] = null; // free asap for GC
-			namespaceSerializer.serialize(toWrite.namespace, dov);
-			keySerializer.serialize(toWrite.key, dov);
-			stateSerializer.serialize(toWrite.state, dov);
+			localNamespaceSerializer.serialize(toWrite.namespace, dov);
+			localKeySerializer.serialize(toWrite.key, dov);
+			localStateSerializer.serialize(toWrite.state, dov);
 		}
 	}
 
@@ -185,4 +203,4 @@ public class CopyOnWriteStateTableSnapshot<K, N, S>
 	boolean isOwner(CopyOnWriteStateTable<K, N, S> stateTable) {
 		return stateTable == owningStateTable;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/89866a5a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
index 08896da..976b9aa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
@@ -23,13 +23,19 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.state.ArrayListSerializer;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StateTransformationFunction;
+import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -37,7 +43,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
-public class CopyOnWriteStateTableTest {
+public class CopyOnWriteStateTableTest extends TestLogger {
 
 	/**
 	 * Testing the basic map operations.
@@ -380,6 +386,77 @@ public class CopyOnWriteStateTableTest {
 		Assert.assertTrue(originalState5 == stateTable.get(5, 1));
 	}
 
+	/**
+	 * This tests that serializers used for snapshots are duplicates of the ones used in
+	 * processing to avoid race conditions in stateful serializers.
+	 */
+	@Test
+	public void testSerializerDuplicationInSnapshot() throws IOException {
+
+		final TestDuplicateSerializer namespaceSerializer = new TestDuplicateSerializer();
+		final TestDuplicateSerializer stateSerializer = new TestDuplicateSerializer();;
+		final TestDuplicateSerializer keySerializer = new TestDuplicateSerializer();;
+
+		RegisteredBackendStateMetaInfo<Integer, Integer> metaInfo =
+			new RegisteredBackendStateMetaInfo<>(
+				StateDescriptor.Type.VALUE,
+				"test",
+				namespaceSerializer,
+				stateSerializer);
+
+		final KeyGroupRange keyGroupRange = new KeyGroupRange(0, 0);
+		InternalKeyContext<Integer> mockKeyContext = new InternalKeyContext<Integer>() {
+			@Override
+			public Integer getCurrentKey() {
+				return 0;
+			}
+
+			@Override
+			public int getCurrentKeyGroupIndex() {
+				return 0;
+			}
+
+			@Override
+			public int getNumberOfKeyGroups() {
+				return 1;
+			}
+
+			@Override
+			public KeyGroupRange getKeyGroupRange() {
+				return keyGroupRange;
+			}
+
+			@Override
+			public TypeSerializer<Integer> getKeySerializer() {
+				return keySerializer;
+			}
+		};
+
+		CopyOnWriteStateTable<Integer, Integer, Integer> table =
+			new CopyOnWriteStateTable<>(mockKeyContext, metaInfo);
+
+		table.put(0, 0, 0, 0);
+		table.put(1, 0, 0, 1);
+		table.put(2, 0, 1, 2);
+
+
+		CopyOnWriteStateTableSnapshot<Integer, Integer, Integer> snapshot = table.createSnapshot();
+
+		try {
+
+			namespaceSerializer.disable();
+			keySerializer.disable();
+			stateSerializer.disable();
+
+			snapshot.writeMappingsInKeyGroup(
+				new DataOutputViewStreamWrapper(
+					new ByteArrayOutputStreamWithPos(1024)), 0);
+
+		} finally {
+			table.releaseSnapshot(snapshot);
+		}
+	}
+
 	@SuppressWarnings("unchecked")
 	private static <K, N, S> Tuple3<K, N, S>[] convert(CopyOnWriteStateTable.StateTableEntry<K, N, S>[] snapshot, int mapSize) {
 
@@ -483,4 +560,94 @@ public class CopyOnWriteStateTableTest {
 			return serializer;
 		}
 	}
-}
\ No newline at end of file
+
+	/**
+	 * Serializer that can be disabled. Duplicates are still enabled, so we can check that
+	 * serializers are duplicated.
+	 */
+	static class TestDuplicateSerializer extends TypeSerializer<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		private static final Integer ZERO = 0;
+
+		private boolean disabled;
+
+		public TestDuplicateSerializer() {
+			this.disabled = false;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public TypeSerializer<Integer> duplicate() {
+			return new TestDuplicateSerializer();
+		}
+
+		@Override
+		public Integer createInstance() {
+			return ZERO;
+		}
+
+		@Override
+		public Integer copy(Integer from) {
+			return from;
+		}
+
+		@Override
+		public Integer copy(Integer from, Integer reuse) {
+			return from;
+		}
+
+		@Override
+		public int getLength() {
+			return 4;
+		}
+
+		@Override
+		public void serialize(Integer record, DataOutputView target) throws IOException {
+			Assert.assertFalse(disabled);
+			target.writeInt(record);
+		}
+
+		@Override
+		public Integer deserialize(DataInputView source) throws IOException {
+			Assert.assertFalse(disabled);
+			return source.readInt();
+		}
+
+		@Override
+		public Integer deserialize(Integer reuse, DataInputView source) throws IOException {
+			Assert.assertFalse(disabled);
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			Assert.assertFalse(disabled);
+			target.writeInt(source.readInt());
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj instanceof TestDuplicateSerializer;
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof TestDuplicateSerializer;
+		}
+
+		@Override
+		public int hashCode() {
+			return getClass().hashCode();
+		}
+
+		public void disable() {
+			this.disabled = true;
+		}
+	}
+}


[23/50] [abbrv] flink git commit: [FLINK-5570] [table] Register ExternalCatalogs in TableEnvironment.

Posted by fh...@apache.org.
[FLINK-5570] [table] Register ExternalCatalogs in TableEnvironment.

This closes #3409.


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

Branch: refs/heads/table-retraction
Commit: 135a57c4bb37eaa9cb85faaff1cc694f9448fabd
Parents: 976e03c
Author: jingzhang <be...@126.com>
Authored: Thu Mar 16 11:24:09 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 20:19:17 2017 +0100

----------------------------------------------------------------------
 docs/dev/table_api.md                           |  37 +++++
 .../flink/table/api/TableEnvironment.scala      | 104 ++++++++++--
 .../org/apache/flink/table/api/exceptions.scala |  62 +++++--
 .../table/catalog/ExternalCatalogSchema.scala   |  14 +-
 .../flink/table/plan/logical/operators.scala    |   4 +-
 .../flink/table/ExternalCatalogTest.scala       | 161 +++++++++++++++++++
 .../catalog/ExternalCatalogSchemaTest.scala     |   5 +-
 7 files changed, 342 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/docs/dev/table_api.md
----------------------------------------------------------------------
diff --git a/docs/dev/table_api.md b/docs/dev/table_api.md
index 03b916c..117f32f 100644
--- a/docs/dev/table_api.md
+++ b/docs/dev/table_api.md
@@ -344,6 +344,43 @@ tableEnvironment.unregisterTable("Customers")
 </div>
 </div>
 
+Registering external Catalogs
+--------------------------------
+
+An external catalog is defined by the `ExternalCatalog` interface and provides information about databases and tables such as their name, schema, statistics, and access information. An `ExternalCatalog` is registered in a `TableEnvironment` as follows: 
+
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight java %}
+// works for StreamExecutionEnvironment identically
+ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+ExternalCatalog customerCatalog = new InMemoryExternalCatalog();
+
+// register the ExternalCatalog customerCatalog
+tableEnv.registerExternalCatalog("Customers", customerCatalog);
+{% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// works for StreamExecutionEnvironment identically
+val env = ExecutionEnvironment.getExecutionEnvironment
+val tableEnv = TableEnvironment.getTableEnvironment(env)
+
+val customerCatalog: ExternalCatalog = new InMemoryExternalCatalog
+
+// register the ExternalCatalog customerCatalog
+tableEnv.registerExternalCatalog("Customers", customerCatalog)
+
+{% endhighlight %}
+</div>
+</div>
+
+Once registered in a `TableEnvironment`, all tables defined in a `ExternalCatalog` can be accessed from Table API or SQL queries by specifying their full path (`catalog`.`database`.`table`).
+
+Currently, Flink provides an `InMemoryExternalCatalog` for demo and testing purposes. However, the `ExternalCatalog` interface can also be used to connect catalogs like HCatalog or Metastore to the Table API.
 
 Table API
 ----------

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 1dda3a8..bb4c3ac 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -46,6 +46,7 @@ import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => Scala
 import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv}
 import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv}
 import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem}
+import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema}
 import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
 import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions}
@@ -60,6 +61,8 @@ import org.apache.flink.table.validate.FunctionCatalog
 import org.apache.flink.types.Row
 
 import _root_.scala.collection.JavaConverters._
+import _root_.scala.collection.mutable.HashMap
+import _root_.scala.annotation.varargs
 
 /**
   * The abstract base class for batch and stream TableEnvironments.
@@ -71,7 +74,7 @@ abstract class TableEnvironment(val config: TableConfig) {
   // the catalog to hold all registered and translated tables
   // we disable caching here to prevent side effects
   private val internalSchema: CalciteSchema = CalciteSchema.createRootSchema(true, false)
-  private val tables: SchemaPlus = internalSchema.plus()
+  private val rootSchema: SchemaPlus = internalSchema.plus()
 
   // Table API/SQL function catalog
   private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
@@ -79,7 +82,7 @@ abstract class TableEnvironment(val config: TableConfig) {
   // the configuration to create a Calcite planner
   private lazy val frameworkConfig: FrameworkConfig = Frameworks
     .newConfigBuilder
-    .defaultSchema(tables)
+    .defaultSchema(rootSchema)
     .parserConfig(getSqlParserConfig)
     .costFactory(new DataSetCostFactory)
     .typeSystem(new FlinkTypeSystem)
@@ -99,6 +102,9 @@ abstract class TableEnvironment(val config: TableConfig) {
   // a counter for unique attribute names
   private[flink] val attrNameCntr: AtomicInteger = new AtomicInteger(0)
 
+  // registered external catalog names -> catalog
+  private val externalCatalogs = new HashMap[String, ExternalCatalog]
+
   /** Returns the table config to define the runtime behavior of the Table API. */
   def getConfig = config
 
@@ -246,6 +252,35 @@ abstract class TableEnvironment(val config: TableConfig) {
   }
 
   /**
+    * Registers an [[ExternalCatalog]] under a unique name in the TableEnvironment's schema.
+    * All tables registered in the [[ExternalCatalog]] can be accessed.
+    *
+    * @param name            The name under which the externalCatalog will be registered
+    * @param externalCatalog The externalCatalog to register
+    */
+  def registerExternalCatalog(name: String, externalCatalog: ExternalCatalog): Unit = {
+    if (rootSchema.getSubSchema(name) != null) {
+      throw new ExternalCatalogAlreadyExistException(name)
+    }
+    this.externalCatalogs.put(name, externalCatalog)
+    // create an external catalog calicte schema, register it on the root schema
+    ExternalCatalogSchema.registerCatalog(rootSchema, name, externalCatalog)
+  }
+
+  /**
+    * Gets a registered [[ExternalCatalog]] by name.
+    *
+    * @param name The name to look up the [[ExternalCatalog]]
+    * @return The [[ExternalCatalog]]
+    */
+  def getRegisteredExternalCatalog(name: String): ExternalCatalog = {
+    this.externalCatalogs.get(name) match {
+      case Some(catalog) => catalog
+      case None => throw new ExternalCatalogNotExistException(name)
+    }
+  }
+
+  /**
     * Registers a [[ScalarFunction]] under a unique name. Replaces already existing
     * user-defined functions under this name.
     */
@@ -254,6 +289,7 @@ abstract class TableEnvironment(val config: TableConfig) {
     checkForInstantiation(function.getClass)
 
     // register in Table API
+
     functionCatalog.registerFunction(name, function.getClass)
 
     // register in SQL API
@@ -341,7 +377,7 @@ abstract class TableEnvironment(val config: TableConfig) {
   protected def replaceRegisteredTable(name: String, table: AbstractTable): Unit = {
 
     if (isRegistered(name)) {
-      tables.add(name, table)
+      rootSchema.add(name, table)
     } else {
       throw new TableException(s"Table \'$name\' is not registered.")
     }
@@ -350,19 +386,55 @@ abstract class TableEnvironment(val config: TableConfig) {
   /**
     * Scans a registered table and returns the resulting [[Table]].
     *
-    * The table to scan must be registered in the [[TableEnvironment]]'s catalog.
+    * A table to scan must be registered in the TableEnvironment. It can be either directly
+    * registered as DataStream, DataSet, or Table or as member of an [[ExternalCatalog]].
+    *
+    * Examples:
     *
-    * @param tableName The name of the table to scan.
-    * @throws ValidationException if no table is registered under the given name.
-    * @return The scanned table.
+    * - Scanning a directly registered table
+    * {{{
+    *   val tab: Table = tableEnv.scan("tableName")
+    * }}}
+    *
+    * - Scanning a table from a registered catalog
+    * {{{
+    *   val tab: Table = tableEnv.scan("catalogName", "dbName", "tableName")
+    * }}}
+    *
+    * @param tablePath The path of the table to scan.
+    * @throws TableException if no table is found using the given table path.
+    * @return The resulting [[Table]].
     */
-  @throws[ValidationException]
-  def scan(tableName: String): Table = {
-    if (isRegistered(tableName)) {
-      new Table(this, CatalogNode(tableName, getRowType(tableName)))
-    } else {
-      throw new TableException(s"Table \'$tableName\' was not found in the registry.")
+  @throws[TableException]
+  @varargs
+  def scan(tablePath: String*): Table = {
+    scanInternal(tablePath.toArray)
+  }
+
+  @throws[TableException]
+  private def scanInternal(tablePath: Array[String]): Table = {
+    require(tablePath != null && !tablePath.isEmpty, "tablePath must not be null or empty.")
+    val schemaPaths = tablePath.slice(0, tablePath.length - 1)
+    val schema = getSchema(schemaPaths)
+    if (schema != null) {
+      val tableName = tablePath(tablePath.length - 1)
+      val table = schema.getTable(tableName)
+      if (table != null) {
+        return new Table(this, CatalogNode(tablePath, table.getRowType(typeFactory)))
+      }
+    }
+    throw new TableException(s"Table \'${tablePath.mkString(".")}\' was not found.")
+  }
+
+  private def getSchema(schemaPath: Array[String]): SchemaPlus = {
+    var schema = rootSchema
+    for (schemaName <- schemaPath) {
+      schema = schema.getSubSchema(schemaName)
+      if (schema == null) {
+        return schema
+      }
     }
+    schema
   }
 
   /**
@@ -416,7 +488,7 @@ abstract class TableEnvironment(val config: TableConfig) {
       throw new TableException(s"Table \'$name\' already exists. " +
         s"Please, choose a different name.")
     } else {
-      tables.add(name, table)
+      rootSchema.add(name, table)
     }
   }
 
@@ -434,11 +506,11 @@ abstract class TableEnvironment(val config: TableConfig) {
     * @return true, if a table is registered under the name, false otherwise.
     */
   protected def isRegistered(name: String): Boolean = {
-    tables.getTableNames.contains(name)
+    rootSchema.getTableNames.contains(name)
   }
 
   protected def getRowType(name: String): RelDataType = {
-    tables.getTable(name).getRowType(typeFactory)
+    rootSchema.getTable(name).getRowType(typeFactory)
   }
 
   /** Returns a unique temporary attribute name. */

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
index 8632436..760cf75 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
@@ -75,34 +75,34 @@ object ValidationException {
 case class UnresolvedException(msg: String) extends RuntimeException(msg)
 
 /**
-  * Exception for operation on a nonexistent table
+  * Exception for an operation on a nonexistent table
   *
   * @param db    database name
   * @param table table name
-  * @param cause
+  * @param cause the cause
   */
 case class TableNotExistException(
     db: String,
     table: String,
     cause: Throwable)
-    extends RuntimeException(s"table $db.$table does not exist!", cause) {
+    extends RuntimeException(s"Table $db.$table does not exist.", cause) {
 
   def this(db: String, table: String) = this(db, table, null)
 
 }
 
 /**
-  * Exception for adding an already existed table
+  * Exception for adding an already existent table
   *
   * @param db    database name
   * @param table table name
-  * @param cause
+  * @param cause the cause
   */
 case class TableAlreadyExistException(
     db: String,
     table: String,
     cause: Throwable)
-    extends RuntimeException(s"table $db.$table already exists!", cause) {
+    extends RuntimeException(s"Table $db.$table already exists.", cause) {
 
   def this(db: String, table: String) = this(db, table, null)
 
@@ -112,56 +112,84 @@ case class TableAlreadyExistException(
   * Exception for operation on a nonexistent database
   *
   * @param db database name
-  * @param cause
+  * @param cause the cause
   */
 case class DatabaseNotExistException(
     db: String,
     cause: Throwable)
-    extends RuntimeException(s"database $db does not exist!", cause) {
+    extends RuntimeException(s"Database $db does not exist.", cause) {
 
   def this(db: String) = this(db, null)
 }
 
 /**
-  * Exception for adding an already existed database
+  * Exception for adding an already existent database
   *
   * @param db database name
-  * @param cause
+  * @param cause the cause
   */
 case class DatabaseAlreadyExistException(
     db: String,
     cause: Throwable)
-    extends RuntimeException(s"database $db already exists!", cause) {
+    extends RuntimeException(s"Database $db already exists.", cause) {
 
   def this(db: String) = this(db, null)
 }
 
 /**
-  * Exception for does not find any matched [[TableSourceConverter]] for a specified table type
+  * Exception for not finding a [[TableSourceConverter]] for a given table type.
   *
   * @param tableType table type
-  * @param cause
+  * @param cause the cause
   */
 case class NoMatchedTableSourceConverterException(
     tableType: String,
     cause: Throwable)
-    extends RuntimeException(s"find no table source converter matched table type $tableType!",
+    extends RuntimeException(s"Could not find a TableSourceConverter for table type $tableType.",
       cause) {
 
   def this(tableType: String) = this(tableType, null)
 }
 
 /**
-  * Exception for find more than one matched [[TableSourceConverter]] for a specified table type
+  * Exception for finding more than one [[TableSourceConverter]] for a given table type.
   *
   * @param tableType table type
-  * @param cause
+  * @param cause the cause
   */
 case class AmbiguousTableSourceConverterException(
     tableType: String,
     cause: Throwable)
-    extends RuntimeException(s"more than one table source converter matched table type $tableType!",
+    extends RuntimeException(s"More than one TableSourceConverter for table type $tableType.",
       cause) {
 
   def this(tableType: String) = this(tableType, null)
 }
+
+/**
+  * Exception for operation on a nonexistent external catalog
+  *
+  * @param catalogName external catalog name
+  * @param cause the cause
+  */
+case class ExternalCatalogNotExistException(
+    catalogName: String,
+    cause: Throwable)
+    extends RuntimeException(s"External catalog $catalogName does not exist.", cause) {
+
+  def this(catalogName: String) = this(catalogName, null)
+}
+
+/**
+  * Exception for adding an already existent external catalog
+  *
+  * @param catalogName external catalog name
+  * @param cause the cause
+  */
+case class ExternalCatalogAlreadyExistException(
+    catalogName: String,
+    cause: Throwable)
+    extends RuntimeException(s"External catalog $catalogName already exists.", cause) {
+
+  def this(catalogName: String) = this(catalogName, null)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala
index e3ed96e..8e010fa 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala
@@ -136,20 +136,18 @@ class ExternalCatalogSchema(
 object ExternalCatalogSchema {
 
   /**
-    * Creates a FlinkExternalCatalogSchema.
+    * Registers an external catalog in a Calcite schema.
     *
-    * @param parentSchema              Parent schema
-    * @param externalCatalogIdentifier External catalog identifier
-    * @param externalCatalog           External catalog object
-    * @return Created schema
+    * @param parentSchema              Parent schema into which the catalog is registered
+    * @param externalCatalogIdentifier Identifier of the external catalog
+    * @param externalCatalog           The external catalog to register
     */
-  def create(
+  def registerCatalog(
       parentSchema: SchemaPlus,
       externalCatalogIdentifier: String,
-      externalCatalog: ExternalCatalog): ExternalCatalogSchema = {
+      externalCatalog: ExternalCatalog): Unit = {
     val newSchema = new ExternalCatalogSchema(externalCatalogIdentifier, externalCatalog)
     val schemaPlusOfNewSchema = parentSchema.add(externalCatalogIdentifier, newSchema)
     newSchema.registerSubSchemas(schemaPlusOfNewSchema)
-    newSchema
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
index 1b5eafb..559bd75 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
@@ -511,7 +511,7 @@ case class Join(
 }
 
 case class CatalogNode(
-    tableName: String,
+    tablePath: Array[String],
     rowType: RelDataType) extends LeafNode {
 
   val output: Seq[Attribute] = rowType.getFieldList.asScala.map { field =>
@@ -519,7 +519,7 @@ case class CatalogNode(
   }
 
   override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    relBuilder.scan(tableName)
+    relBuilder.scan(tablePath.toIterable.asJava)
   }
 
   override def validate(tableEnv: TableEnvironment): LogicalNode = this

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExternalCatalogTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExternalCatalogTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExternalCatalogTest.scala
new file mode 100644
index 0000000..696468d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExternalCatalogTest.scala
@@ -0,0 +1,161 @@
+/*
+ * 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.table
+
+import org.apache.flink.table.utils.{CommonTestData, TableTestBase}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+/**
+  * Test for external catalog query plan.
+  */
+class ExternalCatalogTest extends TableTestBase {
+  private val table1Path: Array[String] = Array("test", "db1", "tb1")
+  private val table1ProjectedFields: Array[String] = Array("a", "b", "c")
+  private val table2Path: Array[String] = Array("test", "db2", "tb2")
+  private val table2ProjectedFields: Array[String] = Array("d", "e", "g")
+
+  @Test
+  def testBatchTableApi(): Unit = {
+    val util = batchTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog)
+
+    val table1 = tEnv.scan("test", "db1", "tb1")
+    val table2 = tEnv.scan("test", "db2", "tb2")
+    val result = table2
+        .select('d * 2, 'e, 'g.upperCase())
+        .unionAll(table1.select('a * 2, 'b, 'c.upperCase()))
+
+    val expected = binaryNode(
+      "DataSetUnion",
+      unaryNode(
+        "DataSetCalc",
+        sourceBatchTableNode(table2Path, table2ProjectedFields),
+        term("select", "*(d, 2) AS _c0", "e", "UPPER(g) AS _c2")
+      ),
+      unaryNode(
+        "DataSetCalc",
+        sourceBatchTableNode(table1Path, table1ProjectedFields),
+        term("select", "*(a, 2) AS _c0", "b", "UPPER(c) AS _c2")
+      ),
+      term("union", "_c0", "e", "_c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testBatchSQL(): Unit = {
+    val util = batchTestUtil()
+
+    util.tEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog)
+
+    val sqlQuery = "SELECT d * 2, e, g FROM test.db2.tb2 WHERE d < 3 UNION ALL " +
+        "(SELECT a * 2, b, c FROM test.db1.tb1)"
+
+    val expected = binaryNode(
+      "DataSetUnion",
+      unaryNode(
+        "DataSetCalc",
+        sourceBatchTableNode(table2Path, table2ProjectedFields),
+        term("select", "*(d, 2) AS EXPR$0", "e", "g"),
+        term("where", "<(d, 3)")),
+      unaryNode(
+        "DataSetCalc",
+        sourceBatchTableNode(table1Path, table1ProjectedFields),
+        term("select", "*(a, 2) AS EXPR$0", "b", "c")
+      ),
+      term("union", "EXPR$0", "e", "g"))
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testStreamTableApi(): Unit = {
+    val util = streamTestUtil()
+    val tEnv = util.tEnv
+
+    util.tEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog)
+
+    val table1 = tEnv.scan("test", "db1", "tb1")
+    val table2 = tEnv.scan("test", "db2", "tb2")
+
+    val result = table2.where("d < 3")
+        .select('d * 2, 'e, 'g.upperCase())
+        .unionAll(table1.select('a * 2, 'b, 'c.upperCase()))
+
+    val expected = binaryNode(
+      "DataStreamUnion",
+      unaryNode(
+        "DataStreamCalc",
+        sourceStreamTableNode(table2Path, table2ProjectedFields),
+        term("select", "*(d, 2) AS _c0", "e", "UPPER(g) AS _c2"),
+        term("where", "<(d, 3)")
+      ),
+      unaryNode(
+        "DataStreamCalc",
+        sourceStreamTableNode(table1Path, table1ProjectedFields),
+        term("select", "*(a, 2) AS _c0", "b", "UPPER(c) AS _c2")
+      ),
+      term("union", "_c0", "e", "_c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testStreamSQL(): Unit = {
+    val util = streamTestUtil()
+
+    util.tEnv.registerExternalCatalog("test", CommonTestData.getInMemoryTestCatalog)
+
+    val sqlQuery = "SELECT d * 2, e, g FROM test.db2.tb2 WHERE d < 3 UNION ALL " +
+        "(SELECT a * 2, b, c FROM test.db1.tb1)"
+
+    val expected = binaryNode(
+      "DataStreamUnion",
+      unaryNode(
+        "DataStreamCalc",
+        sourceStreamTableNode(table2Path, table2ProjectedFields),
+        term("select", "*(d, 2) AS EXPR$0", "e", "g"),
+        term("where", "<(d, 3)")),
+      unaryNode(
+        "DataStreamCalc",
+        sourceStreamTableNode(table1Path, table1ProjectedFields),
+        term("select", "*(a, 2) AS EXPR$0", "b", "c")
+      ),
+      term("union", "EXPR$0", "e", "g"))
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  def sourceBatchTableNode(sourceTablePath: Array[String], fields: Array[String]): String = {
+    s"BatchTableSourceScan(table=[[${sourceTablePath.mkString(", ")}]], " +
+        s"fields=[${fields.mkString(", ")}])"
+  }
+
+  def sourceStreamTableNode(sourceTablePath: Array[String], fields: Array[String]): String = {
+    s"StreamTableSourceScan(table=[[${sourceTablePath.mkString(", ")}]], " +
+        s"fields=[${fields.mkString(", ")}])"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/135a57c4/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala
index 6ffa8c6..b780a3f 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala
@@ -37,7 +37,7 @@ import scala.collection.JavaConverters._
 class ExternalCatalogSchemaTest {
 
   private val schemaName: String = "test"
-  private var externalCatalogSchema: ExternalCatalogSchema = _
+  private var externalCatalogSchema: SchemaPlus = _
   private var calciteCatalogReader: CalciteCatalogReader = _
   private val db = "db1"
   private val tb = "tb1"
@@ -46,7 +46,8 @@ class ExternalCatalogSchemaTest {
   def setUp(): Unit = {
     val rootSchemaPlus: SchemaPlus = CalciteSchema.createRootSchema(true, false).plus()
     val catalog = CommonTestData.getInMemoryTestCatalog
-    externalCatalogSchema = ExternalCatalogSchema.create(rootSchemaPlus, schemaName, catalog)
+    ExternalCatalogSchema.registerCatalog(rootSchemaPlus, schemaName, catalog)
+    externalCatalogSchema = rootSchemaPlus.getSubSchema("schemaName")
     val typeFactory = new FlinkTypeFactory(new FlinkTypeSystem())
     calciteCatalogReader = new CalciteCatalogReader(
       CalciteSchema.from(rootSchemaPlus),


[12/50] [abbrv] flink git commit: [hotfix] Add EvictingWindowOperatorContractTest

Posted by fh...@apache.org.
[hotfix] Add EvictingWindowOperatorContractTest

The existing WindowOperatorContractTest is turned into a test base while
RegularWindowOperatorContract test tests WindowOperator and
EvictingWindowOperatorTest tests EvictingWindowOperator. For this to
work, the base tests now always use List windows and we have specific
tests for reducing/folding windows in RegularWindowOperatorContractTest.

This also patches in the missing side output support for
EvictingWindowOperator.


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

Branch: refs/heads/table-retraction
Commit: 3c4b156527e9ca7cb2dafdda706913e91d688133
Parents: 8319a45
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Mar 21 15:00:24 2017 +0100
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Thu Mar 23 23:29:01 2017 +0800

----------------------------------------------------------------------
 .../windowing/EvictingWindowOperator.java       |  26 +-
 .../operators/windowing/WindowOperator.java     |   4 +-
 .../EvictingWindowOperatorContractTest.java     |  99 ++++++
 .../RegularWindowOperatorContractTest.java      | 288 +++++++++++++++++
 .../windowing/WindowOperatorContractTest.java   | 310 +++++--------------
 5 files changed, 484 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3c4b1565/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 8dfc717..951f661 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
@@ -99,16 +99,15 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 
 	@Override
 	public void processElement(StreamRecord<IN> element) throws Exception {
-		Collection<W> elementWindows = windowAssigner.assignWindows(
-				element.getValue(),
-				element.getTimestamp(),
-				windowAssignerContext);
+		final Collection<W> elementWindows = windowAssigner.assignWindows(
+				element.getValue(), element.getTimestamp(), windowAssignerContext);
 
-		@SuppressWarnings("unchecked")
-		final K key = (K) getKeyedStateBackend().getCurrentKey();
+		//if element is handled by none of assigned elementWindows
+		boolean isSkippedElement = true;
 
-		if (windowAssigner instanceof MergingWindowAssigner) {
+		final K key = this.<K>getKeyedStateBackend().getCurrentKey();
 
+		if (windowAssigner instanceof MergingWindowAssigner) {
 			MergingWindowSet<W> mergingWindows = getMergingWindowSet();
 
 			for (W window : elementWindows) {
@@ -138,11 +137,12 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 							}
 						});
 
-				// check if the window is already inactive
+				// drop if the window is already late
 				if (isWindowLate(actualWindow)) {
 					mergingWindows.retireWindow(actualWindow);
 					continue;
 				}
+				isSkippedElement = false;
 
 				W stateWindow = mergingWindows.getStateWindow(actualWindow);
 				if (stateWindow == null) {
@@ -174,6 +174,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				registerCleanupTimer(actualWindow);
 			}
 
+			// need to make sure to update the merging state in state
 			mergingWindows.persist();
 		} else {
 			for (W window : elementWindows) {
@@ -182,6 +183,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				if (isWindowLate(window)) {
 					continue;
 				}
+				isSkippedElement = false;
 
 				evictingWindowState.setCurrentNamespace(window);
 				evictingWindowState.add(element);
@@ -208,6 +210,14 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window>
 				registerCleanupTimer(window);
 			}
 		}
+
+		// side output input event if
+		// element not handled by any window
+		// late arriving tag has been set
+		// windowAssigner is event time and current timestamp + allowed lateness no less than element timestamp
+		if (isSkippedElement && lateDataOutputTag != null && isElementLate(element)) {
+			sideOutput(element);
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/3c4b1565/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 9ce1ae7..b4283d8 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
@@ -138,7 +138,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 * {@code window.maxTimestamp + allowedLateness} is smaller than the current watermark will
 	 * be emitted to this.
 	 */
-	private final OutputTag<IN> lateDataOutputTag;
+	protected final OutputTag<IN> lateDataOutputTag;
 
 	// ------------------------------------------------------------------------
 	// State that is not checkpointed
@@ -574,7 +574,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 *
 	 * @param element skipped late arriving element to side output
 	 */
-	private void sideOutput(StreamRecord<IN> element){
+	protected void sideOutput(StreamRecord<IN> element){
 		output.collect(lateDataOutputTag, element);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3c4b1565/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorContractTest.java
new file mode 100644
index 0000000..7af4506
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorContractTest.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.util.OutputTag;
+
+/**
+ * These tests verify that {@link EvictingWindowOperator} correctly interacts with the other
+ * windowing components: {@link WindowAssigner},
+ * {@link Trigger}.
+ * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} and window state.
+ *
+ * <p>These tests document the implicit contract that exists between the windowing components.
+ */
+public class EvictingWindowOperatorContractTest extends WindowOperatorContractTest {
+
+	protected <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+			WindowAssigner<Integer, W> assigner,
+			Trigger<Integer, W> trigger,
+			long allowedLatenss,
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction,
+			OutputTag<Integer> lateOutputTag) throws Exception {
+
+		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Integer getKey(Integer value) throws Exception {
+				return value;
+			}
+		};
+
+		ListStateDescriptor<StreamRecord<Integer>> intListDescriptor =
+				new ListStateDescriptor<>(
+						"int-list",
+						(TypeSerializer<StreamRecord<Integer>>) new StreamElementSerializer(IntSerializer.INSTANCE));
+
+		@SuppressWarnings("unchecked")
+		EvictingWindowOperator<Integer, Integer, OUT, W> operator = new EvictingWindowOperator<>(
+				assigner,
+				assigner.getWindowSerializer(new ExecutionConfig()),
+				keySelector,
+				IntSerializer.INSTANCE,
+				intListDescriptor,
+				windowFunction,
+				trigger,
+				CountEvictor.<W>of(100),
+				allowedLatenss,
+				lateOutputTag);
+
+		return new KeyedOneInputStreamOperatorTestHarness<>(
+				operator,
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+	}
+
+	protected <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+			WindowAssigner<Integer, W> assigner,
+			Trigger<Integer, W> trigger,
+			long allowedLatenss,
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction) throws Exception {
+
+		return createWindowOperator(
+				assigner,
+				trigger,
+				allowedLatenss,
+				windowFunction,
+				null /* late output tag */);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c4b1565/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
new file mode 100644
index 0000000..11508c5
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/RegularWindowOperatorContractTest.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.AppendingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.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.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.util.OutputTag;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * These tests verify that {@link WindowOperator} correctly interacts with the other windowing
+ * components: {@link WindowAssigner},
+ * {@link Trigger}.
+ * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} and window state.
+ *
+ * <p>These tests document the implicit contract that exists between the windowing components.
+ */
+public class RegularWindowOperatorContractTest extends WindowOperatorContractTest {
+
+	@Test
+	public void testReducingWindow() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+
+		ReducingStateDescriptor<Integer> intReduceSumDescriptor =
+				new ReducingStateDescriptor<>(
+						"int-reduce",
+						new ReduceFunction<Integer>() {
+							private static final long serialVersionUID = 1L;
+
+							@Override
+							public Integer reduce(Integer a, Integer b) throws Exception {
+								return a + b;
+							}
+						},
+						IntSerializer.INSTANCE);
+
+		final ValueStateDescriptor<String> valueStateDescriptor =
+				new ValueStateDescriptor<>("string-state", StringSerializer.INSTANCE);
+
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intReduceSumDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		// insert two elements without firing
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+	}
+
+	@Test
+	public void testFoldingWindow() throws Exception {
+
+		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
+		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
+		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
+
+		FoldingStateDescriptor<Integer, Integer> intFoldSumDescriptor =
+			new FoldingStateDescriptor<>(
+					"int-fold",
+					0,
+					new FoldFunction<Integer, Integer>() {
+						private static final long serialVersionUID = 1L;
+
+						@Override
+						public Integer fold(Integer accumulator, Integer value) throws Exception {
+							return accumulator + value;
+						}
+					},
+					IntSerializer.INSTANCE);
+
+		final ValueStateDescriptor<String> valueStateDescriptor =
+				new ValueStateDescriptor<>("string-state", StringSerializer.INSTANCE);
+
+		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
+				createWindowOperator(mockAssigner, mockTrigger, 0L, intFoldSumDescriptor, mockWindowFunction);
+
+		testHarness.open();
+
+		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
+				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
+
+		assertEquals(0, testHarness.getOutput().size());
+		assertEquals(0, testHarness.numKeyedStateEntries());
+
+		// insert two elements without firing
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		doAnswer(new Answer<TriggerResult>() {
+			@Override
+			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
+				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
+				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
+				context.registerEventTimeTimer(window.getEnd());
+				context.getPartitionedState(valueStateDescriptor).update("hello");
+				return TriggerResult.FIRE;
+			}
+		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
+
+		testHarness.processElement(new StreamRecord<>(1, 0L));
+
+		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
+
+		// clear is only called at cleanup time/GC time
+		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
+
+		// FIRE should not purge contents
+		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
+		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
+	}
+
+	/**
+	 * Special method for creating a {@link WindowOperator} with a custom {@link StateDescriptor}
+	 * for the window contents state.
+	 */
+	private <W extends Window, ACC, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+			WindowAssigner<Integer, W> assigner,
+			Trigger<Integer, W> trigger,
+			long allowedLatenss,
+			StateDescriptor<? extends AppendingState<Integer, ACC>, ?> stateDescriptor,
+			InternalWindowFunction<ACC, OUT, Integer, W> windowFunction) throws Exception {
+
+		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Integer getKey(Integer value) throws Exception {
+				return value;
+			}
+		};
+
+		@SuppressWarnings("unchecked")
+		WindowOperator<Integer, Integer, ACC, OUT, W> operator = new WindowOperator<>(
+				assigner,
+				assigner.getWindowSerializer(new ExecutionConfig()),
+				keySelector,
+				IntSerializer.INSTANCE,
+				stateDescriptor,
+				windowFunction,
+				trigger,
+				allowedLatenss,
+				null /* late output tag */);
+
+		return new KeyedOneInputStreamOperatorTestHarness<>(
+				operator,
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+	}
+
+	@Override
+	protected <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+			WindowAssigner<Integer, W> assigner,
+			Trigger<Integer, W> trigger,
+			long allowedLatenss,
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction,
+			OutputTag<Integer> lateOutputTag) throws Exception {
+
+		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Integer getKey(Integer value) throws Exception {
+				return value;
+			}
+		};
+
+		ListStateDescriptor<Integer> intListDescriptor =
+				new ListStateDescriptor<>("int-list", IntSerializer.INSTANCE);
+
+
+		@SuppressWarnings("unchecked")
+		WindowOperator<Integer, Integer, Iterable<Integer>, OUT, W> operator = new WindowOperator<>(
+				assigner,
+				assigner.getWindowSerializer(new ExecutionConfig()),
+				keySelector,
+				IntSerializer.INSTANCE,
+				intListDescriptor,
+				windowFunction,
+				trigger,
+				allowedLatenss,
+				lateOutputTag);
+
+		return new KeyedOneInputStreamOperatorTestHarness<>(
+				operator,
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+	}
+
+	@Override
+	protected <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+			WindowAssigner<Integer, W> assigner,
+			Trigger<Integer, W> trigger,
+			long allowedLatenss,
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction) throws Exception {
+
+		return createWindowOperator(
+				assigner,
+				trigger,
+				allowedLatenss,
+				windowFunction,
+				null /* late output tag */);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c4b1565/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
index c542b43..abc7b3e 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorContractTest.java
@@ -18,20 +18,32 @@
 package org.apache.flink.streaming.runtime.operators.windowing;
 
 
+import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.isStreamRecord;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyCollection;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 import com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.AppendingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -42,55 +54,38 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
 import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.OutputTag;
 import org.apache.flink.util.TestLogger;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.verification.VerificationMode;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.flink.streaming.runtime.operators.windowing.StreamRecordMatchers.isStreamRecord;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.*;
-
 /**
- * These tests verify that {@link WindowOperator} correctly interacts with the other windowing
+ * Base for window operator tests that verify correct interaction with the other windowing
  * components: {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner},
  * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
  * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} and window state.
  *
  * <p>These tests document the implicit contract that exists between the windowing components.
  */
-public class WindowOperatorContractTest extends TestLogger {
+public abstract class WindowOperatorContractTest extends TestLogger {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
 
 	private static ValueStateDescriptor<String> valueStateDescriptor =
 			new ValueStateDescriptor<>("string-state", StringSerializer.INSTANCE, null);
 
-	private static ListStateDescriptor<Integer> intListDescriptor =
-			new ListStateDescriptor<>("int-list", IntSerializer.INSTANCE);
-
-	private static ReducingStateDescriptor<Integer> intReduceSumDescriptor =
-			new ReducingStateDescriptor<>("int-reduce", new Sum(), IntSerializer.INSTANCE);
-
-	private static FoldingStateDescriptor<Integer, Integer> intFoldSumDescriptor =
-			new FoldingStateDescriptor<>("int-fold", 0, new FoldSum(), IntSerializer.INSTANCE);
-
 	static <IN, OUT, KEY, W extends Window> InternalWindowFunction<IN, OUT, KEY, W> mockWindowFunction() throws Exception {
 		@SuppressWarnings("unchecked")
 		InternalWindowFunction<IN, OUT, KEY, W> mockWindowFunction = mock(InternalWindowFunction.class);
@@ -313,7 +308,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction, lateOutputTag);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction, lateOutputTag);
 
 		testHarness.open();
 
@@ -338,7 +333,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction, lateOutputTag);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction, lateOutputTag);
 
 		testHarness.open();
 
@@ -376,7 +371,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -401,7 +396,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -425,7 +420,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -472,7 +467,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		OneInputStreamOperatorTestHarness<Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -488,100 +483,6 @@ public class WindowOperatorContractTest extends TestLogger {
 	}
 
 	@Test
-	public void testReducingWindow() throws Exception {
-
-		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
-		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
-		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intReduceSumDescriptor, mockWindowFunction);
-
-		testHarness.open();
-
-		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
-				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
-
-		assertEquals(0, testHarness.getOutput().size());
-		assertEquals(0, testHarness.numKeyedStateEntries());
-
-		// insert two elements without firing
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-
-		doAnswer(new Answer<TriggerResult>() {
-			@Override
-			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
-				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
-				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
-				context.registerEventTimeTimer(window.getEnd());
-				context.getPartitionedState(valueStateDescriptor).update("hello");
-				return TriggerResult.FIRE;
-			}
-		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
-
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-
-		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-
-		// clear is only called at cleanup time/GC time
-		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
-
-		// FIRE should not purge contents
-		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
-		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
-	}
-
-	@Test
-	public void testFoldingWindow() throws Exception {
-
-		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
-		Trigger<Integer, TimeWindow> mockTrigger = mockTrigger();
-		InternalWindowFunction<Integer, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intFoldSumDescriptor, mockWindowFunction);
-
-		testHarness.open();
-
-		when(mockAssigner.assignWindows(anyInt(), anyLong(), anyAssignerContext()))
-				.thenReturn(Arrays.asList(new TimeWindow(2, 4), new TimeWindow(0, 2)));
-
-		assertEquals(0, testHarness.getOutput().size());
-		assertEquals(0, testHarness.numKeyedStateEntries());
-
-		// insert two elements without firing
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-
-		doAnswer(new Answer<TriggerResult>() {
-			@Override
-			public TriggerResult answer(InvocationOnMock invocation) throws Exception {
-				TimeWindow window = (TimeWindow) invocation.getArguments()[2];
-				Trigger.TriggerContext context = (Trigger.TriggerContext) invocation.getArguments()[3];
-				context.registerEventTimeTimer(window.getEnd());
-				context.getPartitionedState(valueStateDescriptor).update("hello");
-				return TriggerResult.FIRE;
-			}
-		}).when(mockTrigger).onElement(Matchers.<Integer>anyObject(), anyLong(), anyTimeWindow(), anyTriggerContext());
-
-		testHarness.processElement(new StreamRecord<>(1, 0L));
-
-		verify(mockWindowFunction, times(2)).apply(eq(1), anyTimeWindow(), anyInt(), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(0, 2)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-		verify(mockWindowFunction, times(1)).apply(eq(1), eq(new TimeWindow(2, 4)), eq(3), WindowOperatorContractTest.<Void>anyCollector());
-
-		// clear is only called at cleanup time/GC time
-		verify(mockTrigger, never()).clear(anyTimeWindow(), anyTriggerContext());
-
-		// FIRE should not purge contents
-		assertEquals(4, testHarness.numKeyedStateEntries()); // window contents plus trigger state
-		assertEquals(4, testHarness.numEventTimeTimers()); // window timers/gc timers
-	}
-
-	@Test
 	public void testEmittingFromWindowFunction() throws Exception {
 
 		WindowAssigner<Integer, TimeWindow> mockAssigner = mockTimeWindowAssigner();
@@ -589,7 +490,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, String, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, String> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -640,7 +541,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, String, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, String> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -683,7 +584,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -725,7 +626,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -768,7 +669,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -813,7 +714,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -866,7 +767,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -924,7 +825,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -986,7 +887,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1047,7 +948,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1115,7 +1016,7 @@ public class WindowOperatorContractTest extends TestLogger {
 				mock(InternalWindowFunction.class);
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, List<Integer>> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1179,7 +1080,7 @@ public class WindowOperatorContractTest extends TestLogger {
 				mock(InternalWindowFunction.class);
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, List<Integer>> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1243,7 +1144,7 @@ public class WindowOperatorContractTest extends TestLogger {
 				mock(InternalWindowFunction.class);
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, List<Integer>> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1313,7 +1214,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1360,7 +1261,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1418,7 +1319,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1459,7 +1360,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1501,7 +1402,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1598,7 +1499,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1681,7 +1582,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1725,7 +1626,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1773,7 +1674,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		assertEquals(Long.MAX_VALUE, GlobalWindow.get().maxTimestamp());
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1798,7 +1699,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1827,7 +1728,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1873,7 +1774,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1918,7 +1819,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -1974,7 +1875,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2044,7 +1945,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2115,7 +2016,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2181,7 +2082,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2209,7 +2110,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2248,7 +2149,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 20L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 20L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2287,7 +2188,7 @@ public class WindowOperatorContractTest extends TestLogger {
 		InternalWindowFunction<Iterable<Integer>, Void, Integer, TimeWindow> mockWindowFunction = mockWindowFunction();
 
 		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Void> testHarness =
-				createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+				createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.open();
 
@@ -2332,11 +2233,11 @@ public class WindowOperatorContractTest extends TestLogger {
 		}).when(mockTrigger).clear(anyTimeWindow(), anyTriggerContext());
 
 		// only fire on the timestamp==0L timers, not the gc timers
-		when(mockTrigger.onEventTime(eq(0L), Matchers.<TimeWindow>any(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
+		when(mockTrigger.onEventTime(eq(0L), anyTimeWindow(), anyTriggerContext())).thenReturn(TriggerResult.FIRE);
 
 		mockWindowFunction = mockWindowFunction();
 
-		testHarness = createWindowOperator(mockAssigner, mockTrigger, 0L, intListDescriptor, mockWindowFunction);
+		testHarness = createWindowOperator(mockAssigner, mockTrigger, 0L, mockWindowFunction);
 
 		testHarness.setup();
 		testHarness.initializeState(snapshot);
@@ -2367,76 +2268,19 @@ public class WindowOperatorContractTest extends TestLogger {
 		assertEquals(0, testHarness.numEventTimeTimers());
 	}
 
-	private <W extends Window, ACC, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+	protected abstract <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
 			WindowAssigner<Integer, W> assigner,
 			Trigger<Integer, W> trigger,
-			long allowedLatenss,
-			StateDescriptor<? extends AppendingState<Integer, ACC>, ?> stateDescriptor,
-			InternalWindowFunction<ACC, OUT, Integer, W> windowFunction,
-			OutputTag<Integer> lateOutputTag) throws Exception {
-
-		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
-			private static final long serialVersionUID = 1L;
+			long allowedLateness,
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction,
+			OutputTag<Integer> lateOutputTag) throws Exception;
 
-			@Override
-			public Integer getKey(Integer value) throws Exception {
-				return value;
-			}
-		};
-
-		@SuppressWarnings("unchecked")
-		WindowOperator<Integer, Integer, ACC, OUT, W> operator = new WindowOperator<>(
-				assigner,
-				assigner.getWindowSerializer(new ExecutionConfig()),
-				keySelector,
-				IntSerializer.INSTANCE,
-				stateDescriptor,
-				windowFunction,
-				trigger,
-				allowedLatenss,
-				lateOutputTag);
-
-		return new KeyedOneInputStreamOperatorTestHarness<>(
-				operator,
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-	}
-
-	private <W extends Window, ACC, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
+	protected abstract <W extends Window, OUT> KeyedOneInputStreamOperatorTestHarness<Integer, Integer, OUT> createWindowOperator(
 			WindowAssigner<Integer, W> assigner,
 			Trigger<Integer, W> trigger,
 			long allowedLatenss,
-			StateDescriptor<? extends AppendingState<Integer, ACC>, ?> stateDescriptor,
-			InternalWindowFunction<ACC, OUT, Integer, W> windowFunction) throws Exception {
-		return createWindowOperator(
-				assigner,
-				trigger,
-				allowedLatenss,
-				stateDescriptor,
-				windowFunction,
-				null /* late output tag */);
-	}
-
+			InternalWindowFunction<Iterable<Integer>, OUT, Integer, W> windowFunction) throws Exception;
 
-	private static class Sum implements ReduceFunction<Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-	}
-
-	private static class FoldSum implements FoldFunction<Integer, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer fold(
-				Integer accumulator,
-				Integer value) throws Exception {
-			return accumulator + value;
-		}
-	}
 
 	private interface TimeDomainAdaptor {
 


[20/50] [abbrv] flink git commit: [FLINK-5658] [table] Add event-time OVER ROWS/RANGE UNBOUNDED PRECEDING aggregation to SQL.

Posted by fh...@apache.org.
[FLINK-5658] [table] Add event-time OVER ROWS/RANGE UNBOUNDED PRECEDING aggregation to SQL.

This closes #3386.


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

Branch: refs/heads/table-retraction
Commit: fe2c61a28e6a5300b2cf4c1e50ee884b51ef42c9
Parents: 7a9d39f
Author: hongyuhong 00223286 <ho...@huawei.com>
Authored: Fri Mar 24 09:31:59 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Mar 24 20:19:17 2017 +0100

----------------------------------------------------------------------
 .../datastream/DataStreamOverAggregate.scala    |  62 +++--
 .../table/runtime/aggregate/AggregateUtil.scala |  70 +++--
 .../UnboundedEventTimeOverProcessFunction.scala | 224 ++++++++++++++++
 .../table/api/scala/stream/sql/SqlITCase.scala  | 263 ++++++++++++++++++-
 .../scala/stream/sql/WindowAggregateTest.scala  |  64 ++++-
 5 files changed, 634 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fe2c61a2/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
index 547c875..3dd7ee2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
@@ -104,7 +104,7 @@ class DataStreamOverAggregate(
       case _: ProcTimeType =>
         // proc-time OVER window
         if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
-          // non-bounded OVER window
+          // unbounded preceding OVER window
           createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS)
         } else if (
           overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
@@ -126,23 +126,15 @@ class DataStreamOverAggregate(
         }
       case _: RowTimeType =>
         // row-time OVER window
-        if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
-          // non-bounded OVER window
-          if (overWindow.isRows) {
-            // ROWS clause unbounded OVER window
-            throw new TableException(
-              "ROWS clause unbounded row-time OVER window no supported yet.")
-          } else {
-            // RANGE clause unbounded OVER window
-            throw new TableException(
-              "RANGE clause unbounded row-time OVER window no supported yet.")
-          }
-        } else if (overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
-            overWindow.upperBound.isCurrentRow) {
+        if (overWindow.lowerBound.isPreceding &&
+              overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
+          // unbounded preceding OVER window
+          createUnboundedAndCurrentRowEventTimeOverWindow(inputDS)
+        } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
           // bounded OVER window
           if (overWindow.isRows) {
             // ROWS clause bounded OVER window
-            createRowsClauseBoundedAndCurrentRowOverWindow(inputDS, true)
+            createRowsClauseBoundedAndCurrentRowOverWindow(inputDS, isRowTimeType = true)
           } else {
             // RANGE clause bounded OVER window
             throw new TableException(
@@ -187,7 +179,7 @@ class DataStreamOverAggregate(
         val processFunction = AggregateUtil.createUnboundedProcessingOverProcessFunction(
           namedAggregates,
           inputType,
-          false)
+          isPartitioned = false)
 
         inputDS
           .process(processFunction).setParallelism(1).setMaxParallelism(1)
@@ -205,7 +197,6 @@ class DataStreamOverAggregate(
     val overWindow: Group = logicWindow.groups.get(0)
     val partitionKeys: Array[Int] = overWindow.keys.toArray
     val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
-    val inputFields = (0 until inputType.getFieldCount).toArray
 
     val precedingOffset =
       getLowerBoundary(logicWindow, overWindow, getInput()) + 1
@@ -216,7 +207,6 @@ class DataStreamOverAggregate(
     val processFunction = AggregateUtil.createRowsClauseBoundedOverProcessFunction(
       namedAggregates,
       inputType,
-      inputFields,
       precedingOffset,
       isRowTimeType
     )
@@ -244,6 +234,42 @@ class DataStreamOverAggregate(
     result
   }
 
+  def createUnboundedAndCurrentRowEventTimeOverWindow(
+    inputDS: DataStream[Row]): DataStream[Row]  = {
+
+    val overWindow: Group = logicWindow.groups.get(0)
+    val partitionKeys: Array[Int] = overWindow.keys.toArray
+    val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates
+
+    // get the output types
+    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
+    val processFunction = AggregateUtil.createUnboundedEventTimeOverProcessFunction(
+      namedAggregates,
+      inputType)
+
+    val result: DataStream[Row] =
+      // partitioned aggregation
+      if (partitionKeys.nonEmpty) {
+        inputDS.keyBy(partitionKeys: _*)
+          .process(processFunction)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataStream[Row]]
+      }
+      // global non-partitioned aggregation
+      else {
+        inputDS.keyBy(new NullByteKeySelector[Row])
+          .process(processFunction)
+          .setParallelism(1)
+          .setMaxParallelism(1)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataStream[Row]]
+      }
+    result
+  }
+
   private def generateNamedAggregates: Seq[CalcitePair[AggregateCall, String]] = {
     val overWindow: Group = logicWindow.groups.get(0)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe2c61a2/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
index 0084ee5..fdac692 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -104,7 +104,6 @@ object AggregateUtil {
   private[flink] def createRowsClauseBoundedOverProcessFunction(
     namedAggregates: Seq[CalcitePair[AggregateCall, String]],
     inputType: RelDataType,
-    inputFields: Array[Int],
     precedingOffset: Long,
     isRowTimeType: Boolean): ProcessFunction[Row, Row] = {
 
@@ -114,26 +113,49 @@ object AggregateUtil {
         inputType,
         needRetraction = true)
 
-    val aggregationStateType: RowTypeInfo =
-      createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
+    val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
+    val inputRowType = FlinkTypeFactory.toInternalRowTypeInfo(inputType).asInstanceOf[RowTypeInfo]
 
-    val inputRowType: RowTypeInfo =
-      createDataSetAggregateBufferDataType(inputFields, Array(), inputType)
+    if (isRowTimeType) {
+      new RowsClauseBoundedOverProcessFunction(
+        aggregates,
+        aggFields,
+        inputType.getFieldCount,
+        aggregationStateType,
+        inputRowType,
+        precedingOffset
+      )
+    } else {
+      throw TableException(
+        "Bounded partitioned proc-time OVER aggregation is not supported yet.")
+    }
+  }
 
-      val processFunction = if (isRowTimeType) {
-        new RowsClauseBoundedOverProcessFunction(
-          aggregates,
-          aggFields,
-          inputType.getFieldCount,
-          aggregationStateType,
-          inputRowType,
-          precedingOffset
-        )
-      } else {
-        throw TableException(
-          "Bounded partitioned proc-time OVER aggregation is not supported yet.")
-      }
-      processFunction
+  /**
+    * Create an [[ProcessFunction]] to evaluate final aggregate value.
+    *
+    * @param namedAggregates List of calls to aggregate functions and their output field names
+    * @param inputType Input row type
+    * @return [[UnboundedEventTimeOverProcessFunction]]
+    */
+  private[flink] def createUnboundedEventTimeOverProcessFunction(
+   namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+   inputType: RelDataType): UnboundedEventTimeOverProcessFunction = {
+
+    val (aggFields, aggregates) =
+      transformToAggregateFunctions(
+        namedAggregates.map(_.getKey),
+        inputType,
+        needRetraction = false)
+
+    val aggregationStateType: RowTypeInfo = createAccumulatorRowType(aggregates)
+
+    new UnboundedEventTimeOverProcessFunction(
+      aggregates,
+      aggFields,
+      inputType.getFieldCount,
+      aggregationStateType,
+      FlinkTypeFactory.toInternalRowTypeInfo(inputType))
   }
 
   /**
@@ -595,7 +617,7 @@ object AggregateUtil {
       // compute preaggregation type
       val preAggFieldTypes = gkeyInFields
         .map(inputType.getFieldList.get(_).getType)
-        .map(FlinkTypeFactory.toTypeInfo) ++ createAccumulatorType(inputType, aggregates)
+        .map(FlinkTypeFactory.toTypeInfo) ++ createAccumulatorType(aggregates)
       val preAggRowType = new RowTypeInfo(preAggFieldTypes: _*)
 
       (
@@ -701,7 +723,7 @@ object AggregateUtil {
 
     val aggResultTypes = namedAggregates.map(a => FlinkTypeFactory.toTypeInfo(a.left.getType))
 
-    val accumulatorRowType = createAccumulatorRowType(inputType, aggregates)
+    val accumulatorRowType = createAccumulatorRowType(aggregates)
     val aggResultRowType = new RowTypeInfo(aggResultTypes: _*)
     val aggFunction = new AggregateAggFunction(aggregates, aggFields)
 
@@ -1029,7 +1051,6 @@ object AggregateUtil {
   }
 
   private def createAccumulatorType(
-      inputType: RelDataType,
       aggregates: Array[TableAggregateFunction[_]]): Seq[TypeInformation[_]] = {
 
     val aggTypes: Seq[TypeInformation[_]] =
@@ -1068,7 +1089,7 @@ object AggregateUtil {
         .map(FlinkTypeFactory.toTypeInfo)
 
     // get all field data types of all intermediate aggregates
-    val aggTypes: Seq[TypeInformation[_]] = createAccumulatorType(inputType, aggregates)
+    val aggTypes: Seq[TypeInformation[_]] = createAccumulatorType(aggregates)
 
     // concat group key types, aggregation types, and window key types
     val allFieldTypes: Seq[TypeInformation[_]] = windowKeyTypes match {
@@ -1079,10 +1100,9 @@ object AggregateUtil {
   }
 
   private def createAccumulatorRowType(
-      inputType: RelDataType,
       aggregates: Array[TableAggregateFunction[_]]): RowTypeInfo = {
 
-    val aggTypes: Seq[TypeInformation[_]] = createAccumulatorType(inputType, aggregates)
+    val aggTypes: Seq[TypeInformation[_]] = createAccumulatorType(aggregates)
 
     new RowTypeInfo(aggTypes: _*)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe2c61a2/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
new file mode 100644
index 0000000..7616ede
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/UnboundedEventTimeOverProcessFunction.scala
@@ -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.table.runtime.aggregate
+
+import java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.types.Row
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.util.{Collector, Preconditions}
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.streaming.api.operators.TimestampedCollector
+import org.apache.flink.table.functions.{Accumulator, AggregateFunction}
+
+
+/**
+  * A ProcessFunction to support unbounded event-time over-window
+  *
+  * @param aggregates the aggregate functions
+  * @param aggFields  the filed index which the aggregate functions use
+  * @param forwardedFieldCount the input fields count
+  * @param intermediateType the intermediate row tye which the state saved
+  * @param inputType the input row tye which the state saved
+  *
+  */
+class UnboundedEventTimeOverProcessFunction(
+    private val aggregates: Array[AggregateFunction[_]],
+    private val aggFields: Array[Int],
+    private val forwardedFieldCount: Int,
+    private val intermediateType: TypeInformation[Row],
+    private val inputType: TypeInformation[Row])
+  extends ProcessFunction[Row, Row]{
+
+  Preconditions.checkNotNull(aggregates)
+  Preconditions.checkNotNull(aggFields)
+  Preconditions.checkArgument(aggregates.length == aggFields.length)
+
+  private var output: Row = _
+  // state to hold the accumulators of the aggregations
+  private var accumulatorState: ValueState[Row] = _
+  // state to hold rows until the next watermark arrives
+  private var rowMapState: MapState[Long, JList[Row]] = _
+  // list to sort timestamps to access rows in timestamp order
+  private var sortedTimestamps: util.LinkedList[Long] = _
+
+
+  override def open(config: Configuration) {
+    output = new Row(forwardedFieldCount + aggregates.length)
+    sortedTimestamps = new util.LinkedList[Long]()
+
+    // initialize accumulator state
+    val accDescriptor: ValueStateDescriptor[Row] =
+      new ValueStateDescriptor[Row]("accumulatorstate", intermediateType)
+    accumulatorState = getRuntimeContext.getState[Row](accDescriptor)
+
+    // initialize row state
+    val rowListTypeInfo: TypeInformation[JList[Row]] = new ListTypeInfo[Row](inputType)
+    val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+      new MapStateDescriptor[Long, JList[Row]]("rowmapstate",
+        BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo)
+    rowMapState = getRuntimeContext.getMapState(mapStateDescriptor)
+  }
+
+  /**
+    * Puts an element from the input stream into state if it is not late.
+    * Registers a timer for the next watermark.
+    *
+    * @param input The input value.
+    * @param ctx   The ctx to register timer or get current time
+    * @param out   The collector for returning result values.
+    *
+    */
+  override def processElement(
+     input: Row,
+     ctx:  ProcessFunction[Row, Row]#Context,
+     out: Collector[Row]): Unit = {
+
+    val timestamp = ctx.timestamp()
+    val curWatermark = ctx.timerService().currentWatermark()
+
+    // discard late record
+    if (timestamp >= curWatermark) {
+      // ensure every key just registers one timer
+      ctx.timerService.registerEventTimeTimer(curWatermark + 1)
+
+      // put row into state
+      var rowList = rowMapState.get(timestamp)
+      if (rowList == null) {
+        rowList = new util.ArrayList[Row]()
+      }
+      rowList.add(input)
+      rowMapState.put(timestamp, rowList)
+    }
+  }
+
+  /**
+    * Called when a watermark arrived.
+    * Sorts records according the timestamp, computes aggregates, and emits all records with
+    * timestamp smaller than the watermark in timestamp order.
+    *
+    * @param timestamp The timestamp of the firing timer.
+    * @param ctx       The ctx to register timer or get current time
+    * @param out       The collector for returning result values.
+    */
+  override def onTimer(
+      timestamp: Long,
+      ctx: ProcessFunction[Row, Row]#OnTimerContext,
+      out: Collector[Row]): Unit = {
+
+    Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[Row]])
+    val collector = out.asInstanceOf[TimestampedCollector[Row]]
+
+    val keyIterator = rowMapState.keys.iterator
+    if (keyIterator.hasNext) {
+      val curWatermark = ctx.timerService.currentWatermark
+      var existEarlyRecord: Boolean = false
+      var i = 0
+
+      // sort the record timestamps
+      do {
+        val recordTime = keyIterator.next
+        // only take timestamps smaller/equal to the watermark
+        if (recordTime <= curWatermark) {
+          insertToSortedList(recordTime)
+        } else {
+          existEarlyRecord = true
+        }
+      } while (keyIterator.hasNext)
+
+      // get last accumulator
+      var lastAccumulator = accumulatorState.value
+      if (lastAccumulator == null) {
+        // initialize accumulator
+        lastAccumulator = new Row(aggregates.length)
+        while (i < aggregates.length) {
+          lastAccumulator.setField(i, aggregates(i).createAccumulator())
+          i += 1
+        }
+      }
+
+      // emit the rows in order
+      while (!sortedTimestamps.isEmpty) {
+        val curTimestamp = sortedTimestamps.removeFirst()
+        val curRowList = rowMapState.get(curTimestamp)
+        collector.setAbsoluteTimestamp(curTimestamp)
+
+        var j = 0
+        while (j < curRowList.size) {
+          val curRow = curRowList.get(j)
+          i = 0
+
+          // copy forwarded fields to output row
+          while (i < forwardedFieldCount) {
+            output.setField(i, curRow.getField(i))
+            i += 1
+          }
+
+          // update accumulators and copy aggregates to output row
+          i = 0
+          while (i < aggregates.length) {
+            val index = forwardedFieldCount + i
+            val accumulator = lastAccumulator.getField(i).asInstanceOf[Accumulator]
+            aggregates(i).accumulate(accumulator, curRow.getField(aggFields(i)))
+            output.setField(index, aggregates(i).getValue(accumulator))
+            i += 1
+          }
+          // emit output row
+          collector.collect(output)
+          j += 1
+        }
+        rowMapState.remove(curTimestamp)
+      }
+
+      accumulatorState.update(lastAccumulator)
+
+      // if are are rows with timestamp > watermark, register a timer for the next watermark
+      if (existEarlyRecord) {
+        ctx.timerService.registerEventTimeTimer(curWatermark + 1)
+      }
+    }
+  }
+
+  /**
+   * Inserts timestamps in order into a linked list.
+   *
+   * If timestamps arrive in order (as in case of using the RocksDB state backend) this is just
+   * an append with O(1).
+   */
+  private def insertToSortedList(recordTimeStamp: Long) = {
+    val listIterator = sortedTimestamps.listIterator(sortedTimestamps.size)
+    var continue = true
+    while (listIterator.hasPrevious && continue) {
+      val timestamp = listIterator.previous
+      if (recordTimeStamp >= timestamp) {
+        listIterator.next
+        listIterator.add(recordTimeStamp)
+        continue = false
+      }
+    }
+
+    if (continue) {
+      sortedTimestamps.addFirst(recordTimeStamp)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe2c61a2/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
index 19350a7..34a68b2 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.table.api.scala.stream.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.streaming.api.functions.source.SourceFunction
 import org.apache.flink.table.api.scala.stream.sql.SqlITCase.EventTimeSourceFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.api.watermark.Watermark
 import org.apache.flink.table.api.{TableEnvironment, TableException}
@@ -436,6 +436,266 @@ class SqlITCase extends StreamingWithStateTestBase {
     env.execute()
   }
 
+  /** test sliding event-time unbounded window with partition by **/
+  @Test
+  def testUnboundedEventTimeRowWindowWithPartition(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 1L, "Hello")),
+      Left(14000002L, (3, 1L, "Hello")),
+      Left(14000003L, (1, 2L, "Hello")),
+      Left(14000004L, (1, 3L, "Hello world")),
+      Left(14000007L, (3, 2L, "Hello world")),
+      Left(14000008L, (2, 2L, "Hello world")),
+      Right(14000010L),
+      // the next 3 elements are late
+      Left(14000008L, (1, 4L, "Hello world")),
+      Left(14000008L, (2, 3L, "Hello world")),
+      Left(14000008L, (3, 3L, "Hello world")),
+      Left(14000012L, (1, 5L, "Hello world")),
+      Right(14000020L),
+      Left(14000021L, (1, 6L, "Hello world")),
+      // the next 3 elements are late
+      Left(14000019L, (1, 6L, "Hello world")),
+      Left(14000018L, (2, 4L, "Hello world")),
+      Left(14000018L, (3, 4L, "Hello world")),
+      Left(14000022L, (2, 5L, "Hello world")),
+      Left(14000022L, (3, 5L, "Hello world")),
+      Left(14000024L, (1, 7L, "Hello world")),
+      Left(14000023L, (1, 8L, "Hello world")),
+      Left(14000021L, (1, 9L, "Hello world")),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,2,Hello,2,1,2,2,2",
+      "1,3,Hello world,5,2,2,3,2",
+      "1,1,Hi,6,3,2,3,1",
+      "2,1,Hello,1,1,1,1,1",
+      "2,2,Hello world,3,2,1,2,1",
+      "3,1,Hello,1,1,1,1,1",
+      "3,2,Hello world,3,2,1,2,1",
+      "1,5,Hello world,11,4,2,5,1",
+      "1,6,Hello world,17,5,3,6,1",
+      "1,9,Hello world,26,6,4,9,1",
+      "1,8,Hello world,34,7,4,9,1",
+      "1,7,Hello world,41,8,5,9,1",
+      "2,5,Hello world,8,3,2,5,1",
+      "3,5,Hello world,8,3,2,5,1"
+    )
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test sliding event-time unbounded window with partition by **/
+  @Test
+  def testUnboundedEventTimeRowWindowWithPartitionMultiThread(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (" +
+      "partition by a order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 1L, "Hello")),
+      Left(14000002L, (3, 1L, "Hello")),
+      Left(14000003L, (1, 2L, "Hello")),
+      Left(14000004L, (1, 3L, "Hello world")),
+      Left(14000007L, (3, 2L, "Hello world")),
+      Left(14000008L, (2, 2L, "Hello world")),
+      Right(14000010L),
+      Left(14000012L, (1, 5L, "Hello world")),
+      Right(14000020L),
+      Left(14000021L, (1, 6L, "Hello world")),
+      Left(14000023L, (2, 5L, "Hello world")),
+      Left(14000024L, (3, 5L, "Hello world")),
+      Left(14000026L, (1, 7L, "Hello world")),
+      Left(14000025L, (1, 8L, "Hello world")),
+      Left(14000022L, (1, 9L, "Hello world")),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,2,Hello,2,1,2,2,2",
+      "1,3,Hello world,5,2,2,3,2",
+      "1,1,Hi,6,3,2,3,1",
+      "2,1,Hello,1,1,1,1,1",
+      "2,2,Hello world,3,2,1,2,1",
+      "3,1,Hello,1,1,1,1,1",
+      "3,2,Hello world,3,2,1,2,1",
+      "1,5,Hello world,11,4,2,5,1",
+      "1,6,Hello world,17,5,3,6,1",
+      "1,9,Hello world,26,6,4,9,1",
+      "1,8,Hello world,34,7,4,9,1",
+      "1,7,Hello world,41,8,5,9,1",
+      "2,5,Hello world,8,3,2,5,1",
+      "3,5,Hello world,8,3,2,5,1"
+    )
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test sliding event-time unbounded window without partitiion by **/
+  @Test
+  def testUnboundedEventTimeRowWindowWithoutPartition(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 2L, "Hello")),
+      Left(14000002L, (3, 5L, "Hello")),
+      Left(14000003L, (1, 3L, "Hello")),
+      Left(14000004L, (3, 7L, "Hello world")),
+      Left(14000007L, (4, 9L, "Hello world")),
+      Left(14000008L, (5, 8L, "Hello world")),
+      Right(14000010L),
+      // this element will be discard because it is late
+      Left(14000008L, (6, 8L, "Hello world")),
+      Right(14000020L),
+      Left(14000021L, (6, 8L, "Hello world")),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "2,2,Hello,2,1,2,2,2",
+      "3,5,Hello,7,2,3,5,2",
+      "1,3,Hello,10,3,3,5,2",
+      "3,7,Hello world,17,4,4,7,2",
+      "1,1,Hi,18,5,3,7,1",
+      "4,9,Hello world,27,6,4,9,1",
+      "5,8,Hello world,35,7,5,9,1",
+      "6,8,Hello world,43,8,5,9,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test sliding event-time unbounded window without partitiion by and arrive early **/
+  @Test
+  def testUnboundedEventTimeRowWindowArriveEarly(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.testResults = mutable.MutableList()
+    env.setParallelism(1)
+
+    val sqlQuery = "SELECT a, b, c, " +
+      "SUM(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "count(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "avg(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "max(b) over (order by rowtime() range between unbounded preceding and current row), " +
+      "min(b) over (order by rowtime() range between unbounded preceding and current row) " +
+      "from T1"
+
+    val data = Seq(
+      Left(14000005L, (1, 1L, "Hi")),
+      Left(14000000L, (2, 2L, "Hello")),
+      Left(14000002L, (3, 5L, "Hello")),
+      Left(14000003L, (1, 3L, "Hello")),
+      // next three elements are early
+      Left(14000012L, (3, 7L, "Hello world")),
+      Left(14000013L, (4, 9L, "Hello world")),
+      Left(14000014L, (5, 8L, "Hello world")),
+      Right(14000010L),
+      Left(14000011L, (6, 8L, "Hello world")),
+      // next element is early
+      Left(14000021L, (6, 8L, "Hello world")),
+      Right(14000020L),
+      Right(14000030L)
+    )
+
+    val t1 = env.addSource(new EventTimeSourceFunction[(Int, Long, String)](data))
+      .toTable(tEnv).as('a, 'b, 'c)
+
+    tEnv.registerTable("T1", t1)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "2,2,Hello,2,1,2,2,2",
+      "3,5,Hello,7,2,3,5,2",
+      "1,3,Hello,10,3,3,5,2",
+      "1,1,Hi,11,4,2,5,1",
+      "6,8,Hello world,19,5,3,8,1",
+      "3,7,Hello world,26,6,4,8,1",
+      "4,9,Hello world,35,7,5,9,1",
+      "5,8,Hello world,43,8,5,9,1",
+      "6,8,Hello world,51,9,5,9,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
 }
 
 object SqlITCase {
@@ -451,5 +711,4 @@ object SqlITCase {
 
     override def cancel(): Unit = ???
   }
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe2c61a2/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
index 9a425b3..7b8b2df 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/WindowAggregateTest.scala
@@ -241,12 +241,67 @@ class WindowAggregateTest extends TableTestBase {
   }
 
   @Test
+  def testUnboundNonPartitionedEventTimeWindowWithRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("orderBy", "ROWTIME"),
+          term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+        ),
+        term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
+  @Test
+  def testUnboundPartitionedEventTimeWindowWithRange() = {
+    val sql = "SELECT " +
+      "c, " +
+      "count(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt1, " +
+      "sum(a) OVER (PARTITION BY c ORDER BY RowTime() RANGE UNBOUNDED preceding) as cnt2 " +
+      "from MyTable"
+
+    val expected =
+      unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamOverAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "a", "c", "ROWTIME() AS $2")
+          ),
+          term("partitionBy", "c"),
+          term("orderBy", "ROWTIME"),
+          term("range", "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"),
+          term("select", "a", "c", "ROWTIME", "COUNT(a) AS w0$o0", "$SUM0(a) AS w0$o1")
+        ),
+        term("select", "c", "w0$o0 AS cnt1", "CASE(>(w0$o0, 0)", "CAST(w0$o1), null) AS cnt2")
+      )
+    streamUtil.verifySql(sql, expected)
+  }
+
+  @Test
   def testBoundPartitionedRowTimeWindowWithRow() = {
     val sql = "SELECT " +
-        "c, " +
-        "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
-        "CURRENT ROW) as cnt1 " +
-        "from MyTable"
+      "c, " +
+      "count(a) OVER (PARTITION BY c ORDER BY RowTime() ROWS BETWEEN 5 preceding AND " +
+      "CURRENT ROW) as cnt1 " +
+      "from MyTable"
 
     val expected =
       unaryNode(
@@ -294,4 +349,5 @@ class WindowAggregateTest extends TableTestBase {
       )
     streamUtil.verifySql(sql, expected)
   }
+
 }


[04/50] [abbrv] flink git commit: [FLINK-3318] Backward compatibility of CEP NFA

Posted by fh...@apache.org.
[FLINK-3318] Backward compatibility of CEP NFA


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

Branch: refs/heads/table-retraction
Commit: d20fb090c31858bc0372a8c84228d796558d56b0
Parents: 9001c4e
Author: Dawid Wysakowicz <da...@getindata.com>
Authored: Tue Mar 21 10:53:07 2017 +0100
Committer: kl0u <kk...@gmail.com>
Committed: Thu Mar 23 10:47:55 2017 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 162 ++++++-
 .../org/apache/flink/cep/nfa/SharedBuffer.java  |  49 ++
 .../java/org/apache/flink/cep/nfa/State.java    |  19 +-
 .../flink/cep/nfa/compiler/NFACompiler.java     | 116 +++++
 .../org/apache/flink/cep/nfa/NFAITCase.java     | 194 +++++++-
 .../cep/operator/CEPMigration12to13Test.java    | 477 +++++++++++++++++++
 .../test/resources/cep-branching-snapshot-1.2   | Bin 0 -> 6736 bytes
 .../resources/cep-single-pattern-snapshot-1.2   | Bin 0 -> 3311 bytes
 .../test/resources/cep-starting-snapshot-1.2    | Bin 0 -> 6526 bytes
 9 files changed, 986 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 3d42248..ab03566 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.cep.nfa;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
 import com.google.common.collect.LinkedHashMultimap;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -25,18 +27,22 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream;
 import org.apache.flink.cep.NonDuplicatingTypeSerializer;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
+import javax.annotation.Nullable;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
+import java.io.OptionalDataException;
 import java.io.Serializable;
+import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -85,9 +91,9 @@ public class NFA<T> implements Serializable {
 	private final NonDuplicatingTypeSerializer<T> nonDuplicatingTypeSerializer;
 
 	/**
-	 * 	Buffer used to store the matched events.
+	 * 	Used only for backward compatibility. Buffer used to store the matched events.
 	 */
-	private final SharedBuffer<String, T> sharedBuffer;
+	private final SharedBuffer<State<T>, T> sharedBuffer = null;
 
 	/**
 	 * A set of all the valid NFA states, as returned by the
@@ -110,12 +116,22 @@ public class NFA<T> implements Serializable {
 	private final boolean handleTimeout;
 
 	/**
+	 * Used only for backward compatibility.
+	 */
+	private int startEventCounter;
+
+	/**
 	 * Current set of {@link ComputationState computation states} within the state machine.
 	 * These are the "active" intermediate states that are waiting for new matching
 	 * events to transition to new valid states.
 	 */
 	private transient Queue<ComputationState<T>> computationStates;
 
+	/**
+	 * 	Buffer used to store the matched events.
+	 */
+	private final SharedBuffer<String, T> stringSharedBuffer;
+
 	public NFA(
 			final TypeSerializer<T> eventSerializer,
 			final long windowTime,
@@ -124,7 +140,7 @@ public class NFA<T> implements Serializable {
 		this.nonDuplicatingTypeSerializer = new NonDuplicatingTypeSerializer<>(eventSerializer);
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
-		sharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
+		stringSharedBuffer = new SharedBuffer<>(nonDuplicatingTypeSerializer);
 		computationStates = new LinkedList<>();
 
 		states = new HashSet<>();
@@ -156,7 +172,7 @@ public class NFA<T> implements Serializable {
 	 * {@code false} otherwise.
 	 */
 	public boolean isEmpty() {
-		return sharedBuffer.isEmpty();
+		return stringSharedBuffer.isEmpty();
 	}
 
 	/**
@@ -194,9 +210,14 @@ public class NFA<T> implements Serializable {
 					}
 				}
 
-				// remove computation state which has exceeded the window length
-				sharedBuffer.release(computationState.getState().getName(), computationState.getEvent(), computationState.getTimestamp());
-				sharedBuffer.remove(computationState.getState().getName(), computationState.getEvent(), computationState.getTimestamp());
+				stringSharedBuffer.release(
+						computationState.getPreviousState().getName(),
+						computationState.getEvent(),
+						computationState.getTimestamp());
+				stringSharedBuffer.remove(
+						computationState.getPreviousState().getName(),
+						computationState.getEvent(),
+						computationState.getTimestamp());
 
 				newComputationStates = Collections.emptyList();
 			} else if (event != null) {
@@ -212,8 +233,8 @@ public class NFA<T> implements Serializable {
 					result.addAll(matches);
 
 					// remove found patterns because they are no longer needed
-					sharedBuffer.release(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
-					sharedBuffer.remove(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
+					stringSharedBuffer.release(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
+					stringSharedBuffer.remove(newComputationState.getPreviousState().getName(), newComputationState.getEvent(), newComputationState.getTimestamp());
 				} else {
 					// add new computation state; it will be processed once the next event arrives
 					computationStates.add(newComputationState);
@@ -230,7 +251,7 @@ public class NFA<T> implements Serializable {
 
 				// remove all elements which are expired
 				// with respect to the window length
-				sharedBuffer.prune(pruningTimestamp);
+				stringSharedBuffer.prune(pruningTimestamp);
 			}
 		}
 
@@ -244,7 +265,7 @@ public class NFA<T> implements Serializable {
 			NFA<T> other = (NFA<T>) obj;
 
 			return nonDuplicatingTypeSerializer.equals(other.nonDuplicatingTypeSerializer) &&
-				sharedBuffer.equals(other.sharedBuffer) &&
+				stringSharedBuffer.equals(other.stringSharedBuffer) &&
 				states.equals(other.states) &&
 				windowTime == other.windowTime;
 		} else {
@@ -254,7 +275,7 @@ public class NFA<T> implements Serializable {
 
 	@Override
 	public int hashCode() {
-		return Objects.hash(nonDuplicatingTypeSerializer, sharedBuffer, states, windowTime);
+		return Objects.hash(nonDuplicatingTypeSerializer, stringSharedBuffer, states, windowTime);
 	}
 
 	private static <T> boolean isEquivalentState(final State<T> s1, final State<T> s2) {
@@ -376,7 +397,7 @@ public class NFA<T> implements Serializable {
 								computationState.getStartTimestamp()
 							)
 						);
-						sharedBuffer.lock(
+						stringSharedBuffer.lock(
 							computationState.getPreviousState().getName(),
 							computationState.getEvent(),
 							computationState.getTimestamp());
@@ -397,14 +418,14 @@ public class NFA<T> implements Serializable {
 					final long startTimestamp;
 					if (computationState.isStartState()) {
 						startTimestamp = timestamp;
-						sharedBuffer.put(
+						stringSharedBuffer.put(
 							consumingState.getName(),
 							event,
 							timestamp,
 							currentVersion);
 					} else {
 						startTimestamp = computationState.getStartTimestamp();
-						sharedBuffer.put(
+						stringSharedBuffer.put(
 							consumingState.getName(),
 							event,
 							timestamp,
@@ -415,7 +436,7 @@ public class NFA<T> implements Serializable {
 					}
 
 					// a new computation state is referring to the shared entry
-					sharedBuffer.lock(consumingState.getName(), event, timestamp);
+					stringSharedBuffer.lock(consumingState.getName(), event, timestamp);
 
 					resultingComputationStates.add(ComputationState.createState(
 						newState,
@@ -429,7 +450,7 @@ public class NFA<T> implements Serializable {
 					//check if newly created state is optional (have a PROCEED path to Final state)
 					final State<T> finalState = findFinalStateAfterProceed(newState, event);
 					if (finalState != null) {
-						sharedBuffer.lock(consumingState.getName(), event, timestamp);
+						stringSharedBuffer.lock(consumingState.getName(), event, timestamp);
 						resultingComputationStates.add(ComputationState.createState(
 							finalState,
 							consumingState,
@@ -450,12 +471,12 @@ public class NFA<T> implements Serializable {
 
 		if (computationState.getEvent() != null) {
 			// release the shared entry referenced by the current computation state.
-			sharedBuffer.release(
+			stringSharedBuffer.release(
 				computationState.getPreviousState().getName(),
 				computationState.getEvent(),
 				computationState.getTimestamp());
 			// try to remove unnecessary shared buffer entries
-			sharedBuffer.remove(
+			stringSharedBuffer.remove(
 				computationState.getPreviousState().getName(),
 				computationState.getEvent(),
 				computationState.getTimestamp());
@@ -546,7 +567,7 @@ public class NFA<T> implements Serializable {
 	 * @return Collection of event sequences which end in the given computation state
 	 */
 	private Collection<Map<String, T>> extractPatternMatches(final ComputationState<T> computationState) {
-		Collection<LinkedHashMultimap<String, T>> paths = sharedBuffer.extractPatterns(
+		Collection<LinkedHashMultimap<String, T>> paths = stringSharedBuffer.extractPatterns(
 			computationState.getPreviousState().getName(),
 			computationState.getEvent(),
 			computationState.getTimestamp(),
@@ -592,6 +613,8 @@ public class NFA<T> implements Serializable {
 		nonDuplicatingTypeSerializer.clearReferences();
 	}
 
+	private final static String BEGINNING_STATE_NAME = "$beginningState$";
+
 	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
 		ois.defaultReadObject();
 
@@ -599,15 +622,103 @@ public class NFA<T> implements Serializable {
 
 		computationStates = new LinkedList<>();
 
+		final List<ComputationState<T>> readComputationStates = new ArrayList<>(numberComputationStates);
+
+		boolean afterMigration = false;
 		for (int i = 0; i < numberComputationStates; i++) {
 			ComputationState<T> computationState = readComputationState(ois);
+			if (computationState.getState().getName().equals(BEGINNING_STATE_NAME)) {
+				afterMigration = true;
+			}
 
-			computationStates.offer(computationState);
+			readComputationStates.add(computationState);
+		}
+
+		if (afterMigration && !readComputationStates.isEmpty()) {
+			try {
+				//Backwards compatibility
+				this.computationStates.addAll(migrateNFA(readComputationStates));
+				final Field newSharedBufferField = NFA.class.getDeclaredField("stringSharedBuffer");
+				final Field sharedBufferField = NFA.class.getDeclaredField("sharedBuffer");
+				sharedBufferField.setAccessible(true);
+				newSharedBufferField.setAccessible(true);
+				newSharedBufferField.set(this, SharedBuffer.migrateSharedBuffer(this.sharedBuffer));
+				sharedBufferField.set(this, null);
+				sharedBufferField.setAccessible(false);
+				newSharedBufferField.setAccessible(false);
+			} catch (Exception e) {
+				throw new IllegalStateException("Could not migrate from earlier version", e);
+			}
+		} else {
+			this.computationStates.addAll(readComputationStates);
 		}
 
 		nonDuplicatingTypeSerializer.clearReferences();
 	}
 
+	/**
+	 * Needed for backward compatibility. First migrates the {@link State} graph see {@link NFACompiler#migrateGraph(State)}.
+	 * Than recreates the {@link ComputationState}s with the new {@link State} graph.
+	 * @param readStates computation states read from snapshot
+	 * @return collection of migrated computation states
+	 */
+	private Collection<ComputationState<T>> migrateNFA(Collection<ComputationState<T>> readStates) {
+		final ArrayList<ComputationState<T>> computationStates = new ArrayList<>();
+
+		final State<T> startState = Iterators.find(
+			readStates.iterator(),
+			new Predicate<ComputationState<T>>() {
+				@Override
+				public boolean apply(@Nullable ComputationState<T> input) {
+					return input != null && input.getState().getName().equals(BEGINNING_STATE_NAME);
+				}
+			}).getState();
+
+		final Map<String, State<T>> convertedStates = NFACompiler.migrateGraph(startState);
+
+		for (ComputationState<T> readState : readStates) {
+			if (!readState.isStartState()) {
+				final String previousName = readState.getState().getName();
+				final String currentName = Iterators.find(
+					readState.getState().getStateTransitions().iterator(),
+					new Predicate<StateTransition<T>>() {
+						@Override
+						public boolean apply(@Nullable StateTransition<T> input) {
+							return input != null && input.getAction() == StateTransitionAction.TAKE;
+						}
+					}).getTargetState().getName();
+
+
+				final State<T> previousState = convertedStates.get(previousName);
+
+				computationStates.add(ComputationState.createState(
+					convertedStates.get(currentName),
+					previousState,
+					readState.getEvent(),
+					readState.getTimestamp(),
+					readState.getVersion(),
+					readState.getStartTimestamp()
+				));
+			}
+		}
+
+		final String startName = Iterators.find(convertedStates.values().iterator(), new Predicate<State<T>>() {
+			@Override
+			public boolean apply(@Nullable State<T> input) {
+				return input != null && input.isStart();
+			}
+		}).getName();
+
+		computationStates.add(ComputationState.createStartState(
+			convertedStates.get(startName),
+			new DeweyNumber(this.startEventCounter)));
+
+		this.states.clear();
+		this.states.addAll(convertedStates.values());
+
+		return computationStates;
+	}
+
 	private void writeComputationState(final ComputationState<T> computationState, final ObjectOutputStream oos) throws IOException {
 		oos.writeObject(computationState.getState());
 		oos.writeObject(computationState.getPreviousState());
@@ -629,7 +740,13 @@ public class NFA<T> implements Serializable {
 	@SuppressWarnings("unchecked")
 	private ComputationState<T> readComputationState(ObjectInputStream ois) throws IOException, ClassNotFoundException {
 		final State<T> state = (State<T>)ois.readObject();
-		final State<T> previousState = (State<T>)ois.readObject();
+		State<T> previousState;
+		try {
+			previousState = (State<T>)ois.readObject();
+		} catch (OptionalDataException e) {
+			previousState = null;
+		}
+
 		final long timestamp = ois.readLong();
 		final DeweyNumber version = (DeweyNumber)ois.readObject();
 		final long startTimestamp = ois.readLong();
@@ -647,6 +764,7 @@ public class NFA<T> implements Serializable {
 		return ComputationState.createState(state, previousState, event, timestamp, version, startTimestamp);
 	}
 
+
 	/**
 	 * Generates a state name from a given name template and an index.
 	 * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index e6a8c75..d5b7876 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -20,6 +20,7 @@ package org.apache.flink.cep.nfa;
 
 import com.google.common.collect.LinkedHashMultimap;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
@@ -463,6 +464,54 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 		}
 	}
 
+	private SharedBuffer(
+		TypeSerializer<V> valueSerializer,
+		Map<K, SharedBufferPage<K, V>> pages) {
+		this.valueSerializer = valueSerializer;
+		this.pages = pages;
+	}
+
+	/**
+	 * For backward compatibility only. Previously the key in {@link SharedBuffer} was {@link State}.
+	 * Now it is {@link String}.
+	 */
+	@Internal
+	static <T> SharedBuffer<String, T> migrateSharedBuffer(SharedBuffer<State<T>, T> buffer) {
+
+		final Map<String, SharedBufferPage<String, T>> pageMap = new HashMap<>();
+		final Map<SharedBufferEntry<State<T>, T>, SharedBufferEntry<String, T>> entries = new HashMap<>();
+
+		for (Map.Entry<State<T>, SharedBufferPage<State<T>, T>> page : buffer.pages.entrySet()) {
+			final SharedBufferPage<String, T> newPage = new SharedBufferPage<>(page.getKey().getName());
+			pageMap.put(newPage.getKey(), newPage);
+
+			for (Map.Entry<ValueTimeWrapper<T>, SharedBufferEntry<State<T>, T>> pageEntry : page.getValue().entries.entrySet()) {
+				final SharedBufferEntry<String, T> newSharedBufferEntry = new SharedBufferEntry<>(
+					pageEntry.getKey(),
+					newPage);
+				newSharedBufferEntry.referenceCounter = pageEntry.getValue().referenceCounter;
+				entries.put(pageEntry.getValue(), newSharedBufferEntry);
+				newPage.entries.put(pageEntry.getKey(), newSharedBufferEntry);
+			}
+		}
+
+		for (Map.Entry<State<T>, SharedBufferPage<State<T>, T>> page : buffer.pages.entrySet()) {
+			for (Map.Entry<ValueTimeWrapper<T>, SharedBufferEntry<State<T>, T>> pageEntry : page.getValue().entries.entrySet()) {
+				final SharedBufferEntry<String, T> newEntry = entries.get(pageEntry.getValue());
+				for (SharedBufferEdge<State<T>, T> edge : pageEntry.getValue().edges) {
+					final SharedBufferEntry<String, T> targetNewEntry = entries.get(edge.getTarget());
+
+					final SharedBufferEdge<String, T> newEdge = new SharedBufferEdge<>(
+						targetNewEntry,
+						edge.getVersion());
+					newEntry.edges.add(newEdge);
+				}
+			}
+		}
+
+		return new SharedBuffer<>(buffer.valueSerializer, pageMap);
+	}
+
 	private SharedBufferEntry<K, V> get(
 		final K key,
 		final V value,

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
index 7bcb6ea..27e0dcd 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/State.java
@@ -20,9 +20,12 @@ package org.apache.flink.cep.nfa;
 
 import org.apache.flink.api.common.functions.FilterFunction;
 
+import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Objects;
 
 /**
@@ -62,7 +65,6 @@ public class State<T> implements Serializable {
 		return stateTransitions;
 	}
 
-
 	private void addStateTransition(
 		final StateTransitionAction action,
 		final State<T> targetState,
@@ -132,4 +134,19 @@ public class State<T> implements Serializable {
 		Final, // the state is a final state for the NFA
 		Normal // the state is neither a start nor a final state
 	}
+
+	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+		ois.defaultReadObject();
+
+		//Backward compatibility. Previous version of StateTransition did not have source state
+		if (!stateTransitions.isEmpty() && stateTransitions.iterator().next().getSourceState() == null) {
+			final List<StateTransition<T>> tmp = new ArrayList<>();
+			tmp.addAll(this.stateTransitions);
+
+			this.stateTransitions.clear();
+			for (StateTransition<T> transition : tmp) {
+				addStateTransition(transition.getAction(), transition.getTargetState(), transition.getCondition());
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index b476c49..8bd8612 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -18,10 +18,15 @@
 
 package org.apache.flink.cep.nfa.compiler;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.State;
+import org.apache.flink.cep.nfa.StateTransition;
+import org.apache.flink.cep.nfa.StateTransitionAction;
 import org.apache.flink.cep.pattern.FilterFunctions;
 import org.apache.flink.cep.pattern.FollowedByPattern;
 import org.apache.flink.cep.pattern.MalformedPatternException;
@@ -31,12 +36,15 @@ import org.apache.flink.cep.pattern.Quantifier;
 import org.apache.flink.cep.pattern.Quantifier.QuantifierProperty;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
+import javax.annotation.Nullable;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -363,6 +371,114 @@ public class NFACompiler {
 	}
 
 	/**
+	 * Used for migrating CEP graphs prior to 1.3. It removes the dummy start, adds the dummy end, and translates all
+	 * states to consuming ones by moving all TAKEs and IGNOREs to the next state. This method assumes each state
+	 * has at most one TAKE and one IGNORE and name of each state is unique. No PROCEED transition is allowed!
+	 *
+	 * @param oldStartState dummy start state of old graph
+	 * @param <T> type of events
+	 * @return map of new states, where key is the name of a state and value is the state itself
+	 */
+	@Internal
+	public static <T> Map<String, State<T>> migrateGraph(State<T> oldStartState) {
+		State<T> oldFirst = oldStartState;
+		State<T> oldSecond = oldStartState.getStateTransitions().iterator().next().getTargetState();
+
+		StateTransition<T> oldFirstToSecondTake = Iterators.find(
+			oldFirst.getStateTransitions().iterator(),
+			new Predicate<StateTransition<T>>() {
+				@Override
+				public boolean apply(@Nullable StateTransition<T> input) {
+					return input != null && input.getAction() == StateTransitionAction.TAKE;
+				}
+
+			});
+
+		StateTransition<T> oldFirstIgnore = Iterators.find(
+			oldFirst.getStateTransitions().iterator(),
+			new Predicate<StateTransition<T>>() {
+				@Override
+				public boolean apply(@Nullable StateTransition<T> input) {
+					return input != null && input.getAction() == StateTransitionAction.IGNORE;
+				}
+
+			}, null);
+
+		StateTransition<T> oldSecondToThirdTake = Iterators.find(
+			oldSecond.getStateTransitions().iterator(),
+			new Predicate<StateTransition<T>>() {
+				@Override
+				public boolean apply(@Nullable StateTransition<T> input) {
+					return input != null && input.getAction() == StateTransitionAction.TAKE;
+				}
+
+			}, null);
+
+		final Map<String, State<T>> convertedStates = new HashMap<>();
+		State<T> newSecond;
+		State<T> newFirst = new State<>(oldSecond.getName(), State.StateType.Start);
+		convertedStates.put(newFirst.getName(), newFirst);
+		while (oldSecondToThirdTake != null) {
+
+			newSecond = new State<T>(oldSecondToThirdTake.getTargetState().getName(), State.StateType.Normal);
+			convertedStates.put(newSecond.getName(), newSecond);
+			newFirst.addTake(newSecond, oldFirstToSecondTake.getCondition());
+
+			if (oldFirstIgnore != null) {
+				newFirst.addIgnore(oldFirstIgnore.getCondition());
+			}
+
+			oldFirst = oldSecond;
+
+			oldFirstToSecondTake = Iterators.find(
+				oldFirst.getStateTransitions().iterator(),
+				new Predicate<StateTransition<T>>() {
+					@Override
+					public boolean apply(@Nullable StateTransition<T> input) {
+						return input != null && input.getAction() == StateTransitionAction.TAKE;
+					}
+
+				});
+
+			oldFirstIgnore = Iterators.find(
+				oldFirst.getStateTransitions().iterator(),
+				new Predicate<StateTransition<T>>() {
+					@Override
+					public boolean apply(@Nullable StateTransition<T> input) {
+						return input != null && input.getAction() == StateTransitionAction.IGNORE;
+					}
+
+				}, null);
+
+			oldSecond = oldSecondToThirdTake.getTargetState();
+
+			oldSecondToThirdTake = Iterators.find(
+				oldSecond.getStateTransitions().iterator(),
+				new Predicate<StateTransition<T>>() {
+					@Override
+					public boolean apply(@Nullable StateTransition<T> input) {
+						return input != null && input.getAction() == StateTransitionAction.TAKE;
+					}
+
+				}, null);
+
+			newFirst = newSecond;
+		}
+
+		final State<T> endingState = new State<>(ENDING_STATE_NAME, State.StateType.Final);
+
+		newFirst.addTake(endingState, oldFirstToSecondTake.getCondition());
+
+		if (oldFirstIgnore != null) {
+			newFirst.addIgnore(oldFirstIgnore.getCondition());
+		}
+
+		convertedStates.put(endingState.getName(), endingState);
+
+		return convertedStates;
+	}
+
+	/**
 	 * Factory interface for {@link NFA}.
 	 *
 	 * @param <T> Type of the input events which are processed by the NFA

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 825ba957..5b05f19 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -39,6 +39,7 @@ import java.util.Map;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class NFAITCase extends TestLogger {
 
@@ -421,7 +422,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testComplexBranchingAfterKleeneStar() {
+	public void testComplexBranchingAfterZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -519,7 +520,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testKleeneStar() {
+	public void testZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -581,7 +582,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEagerKleeneStar() {
+	public void testEagerZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -646,7 +647,7 @@ public class NFAITCase extends TestLogger {
 
 
 	@Test
-	public void testBeginWithKleeneStar() {
+	public void testBeginWithZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event middleEvent1 = new Event(40, "a", 2.0);
@@ -704,7 +705,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testKleeneStarAfterKleeneStar() {
+	public void testZeroOrMoreAfterZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -779,7 +780,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testKleeneStarAfterBranching() {
+	public void testZeroOrMoreAfterBranching() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -865,7 +866,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityNoResultsAfterKleeneStar() {
+	public void testStrictContinuityNoResultsAfterZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event start = new Event(40, "d", 2.0);
@@ -923,7 +924,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityResultsAfterKleeneStar() {
+	public void testStrictContinuityResultsAfterZeroOrMore() {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event start = new Event(40, "d", 2.0);
@@ -1663,4 +1664,181 @@ public class NFAITCase extends TestLogger {
 		), resultingPatterns);
 	}
 
+	/**
+	 * Clearing SharedBuffer
+	 */
+
+	@Test
+	public void testTimesClearingBuffer() {
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).next("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).times(2).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		}).within(Time.milliseconds(8));
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		nfa.process(startEvent, 1);
+		nfa.process(middleEvent1, 2);
+		nfa.process(middleEvent2, 3);
+		nfa.process(middleEvent3, 4);
+		nfa.process(end1, 6);
+
+		//pruning element
+		nfa.process(null, 10);
+
+		assertEquals(true, nfa.isEmpty());
+	}
+
+	@Test
+	public void testOptionalClearingBuffer() {
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).optional().followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		}).within(Time.milliseconds(8));
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		nfa.process(startEvent, 1);
+		nfa.process(middleEvent, 5);
+		nfa.process(end1, 6);
+
+		//pruning element
+		nfa.process(null, 10);
+
+		assertEquals(true, nfa.isEmpty());
+	}
+
+	@Test
+	public void testAtLeastOneClearingBuffer() {
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		}).within(Time.milliseconds(8));
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		nfa.process(startEvent, 1);
+		nfa.process(middleEvent1, 3);
+		nfa.process(middleEvent2, 4);
+		nfa.process(end1, 6);
+
+		//pruning element
+		nfa.process(null, 10);
+
+		assertEquals(true, nfa.isEmpty());
+	}
+
+
+	@Test
+	public void testZeroOrMoreClearingBuffer() {
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event end1 = new Event(44, "b", 5.0);
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedBy("middle").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).zeroOrMore(false).followedBy("end1").where(new FilterFunction<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		}).within(Time.milliseconds(8));
+
+		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+
+		nfa.process(startEvent, 1);
+		nfa.process(middleEvent1, 3);
+		nfa.process(middleEvent2, 4);
+		nfa.process(end1, 6);
+
+		//pruning element
+		nfa.process(null, 10);
+
+		assertEquals(true, nfa.isEmpty());
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
new file mode 100644
index 0000000..65fa733
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
@@ -0,0 +1,477 @@
+/*
+ * 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.cep.operator;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.cep.Event;
+import org.apache.flink.cep.SubEvent;
+import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.junit.Test;
+
+import java.io.FileInputStream;
+import java.io.ObjectInputStream;
+import java.net.URL;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class CEPMigration12to13Test {
+
+	private static String getResourceFilename(String filename) {
+		ClassLoader cl = CEPMigration12to13Test.class.getClassLoader();
+		URL resource = cl.getResource(filename);
+		if (resource == null) {
+			throw new NullPointerException("Missing snapshot resource.");
+		}
+		return resource.getFile();
+	}
+
+	@Test
+	public void testMigrationAfterBranchingPattern() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
+		final Event endEvent = new Event(42, "end", 1.0);
+
+		// uncomment these lines for regenerating the snapshot on Flink 1.2
+//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+//			new KeyedOneInputStreamOperatorTestHarness<>(
+//				new KeyedCEPPatternOperator<>(
+//					Event.createTypeSerializer(),
+//					false,
+//					keySelector,
+//					IntSerializer.INSTANCE,
+//					new NFAFactory(),
+//					true),
+//				keySelector,
+//				BasicTypeInfo.INT_TYPE_INFO);
+//
+//		harness.setup();
+//		harness.open();
+//		harness.processElement(new StreamRecord<Event>(startEvent, 1));
+//		harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
+//		harness.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
+//		harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
+//		harness.processElement(new StreamRecord<Event>(middleEvent2, 3));
+//		harness.processWatermark(new Watermark(5));
+//		// simulate snapshot/restore with empty element queue but NFA state
+//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
+//		FileOutputStream out = new FileOutputStream(
+//				"src/test/resources/cep-branching-snapshot-1.2");
+//		ObjectOutputStream oos = new ObjectOutputStream(out);
+//		oos.writeObject(snapshot.getOperatorChainIndex());
+//		oos.writeObject(snapshot.getLegacyOperatorState());
+//		oos.writeObject(snapshot.getManagedKeyedState());
+//		oos.writeObject(snapshot.getRawKeyedState());
+//		oos.writeObject(snapshot.getManagedOperatorState());
+//		oos.writeObject(snapshot.getRawOperatorState());
+//		out.close();
+//		harness.close();
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new NFAFactory(),
+					true),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		harness.setup();
+		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
+			"cep-branching-snapshot-1.2")));
+		final OperatorStateHandles snapshot = new OperatorStateHandles(
+			(int) ois.readObject(),
+			(StreamStateHandle) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject()
+		);
+		harness.initializeState(snapshot);
+		harness.open();
+
+		harness.processElement(new StreamRecord<>(new Event(42, "start", 1.0), 4));
+		harness.processElement(new StreamRecord<>(endEvent, 5));
+
+		harness.processWatermark(new Watermark(20));
+
+		ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+		// watermark and 2 results
+		assertEquals(3, result.size());
+
+		Object resultObject1 = result.poll();
+		assertTrue(resultObject1 instanceof StreamRecord);
+		StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
+		assertTrue(resultRecord1.getValue() instanceof Map);
+
+		Object resultObject2 = result.poll();
+		assertTrue(resultObject2 instanceof StreamRecord);
+		StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
+		assertTrue(resultRecord2.getValue() instanceof Map);
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap1 = (Map<String, Event>) resultRecord1.getValue();
+
+		assertEquals(startEvent, patternMap1.get("start"));
+		assertEquals(middleEvent1, patternMap1.get("middle"));
+		assertEquals(endEvent, patternMap1.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap2 = (Map<String, Event>) resultRecord2.getValue();
+
+		assertEquals(startEvent, patternMap2.get("start"));
+		assertEquals(middleEvent2, patternMap2.get("middle"));
+		assertEquals(endEvent, patternMap2.get("end"));
+
+		harness.close();
+	}
+
+	@Test
+	public void testStartingNewPatternAfterMigration() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+		final Event startEvent2 = new Event(42, "start", 5.0);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
+		final Event endEvent = new Event(42, "end", 1.0);
+
+		// uncomment these lines for regenerating the snapshot on Flink 1.2
+//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+//			new KeyedOneInputStreamOperatorTestHarness<>(
+//				new KeyedCEPPatternOperator<>(
+//					Event.createTypeSerializer(),
+//					false,
+//					keySelector,
+//					IntSerializer.INSTANCE,
+//					new NFAFactory(),
+//					true),
+//				keySelector,
+//				BasicTypeInfo.INT_TYPE_INFO);
+//
+//		harness.setup();
+//		harness.open();
+//		harness.processElement(new StreamRecord<Event>(startEvent1, 1));
+//		harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
+//		harness.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
+//		harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
+//		harness.processWatermark(new Watermark(5));
+//		// simulate snapshot/restore with empty element queue but NFA state
+//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
+//		FileOutputStream out = new FileOutputStream(
+//				"src/test/resources/cep-starting-snapshot-1.2");
+//		ObjectOutputStream oos = new ObjectOutputStream(out);
+//		oos.writeObject(snapshot.getOperatorChainIndex());
+//		oos.writeObject(snapshot.getLegacyOperatorState());
+//		oos.writeObject(snapshot.getManagedKeyedState());
+//		oos.writeObject(snapshot.getRawKeyedState());
+//		oos.writeObject(snapshot.getManagedOperatorState());
+//		oos.writeObject(snapshot.getRawOperatorState());
+//		out.close();
+//		harness.close();
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new NFAFactory(),
+					true),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		harness.setup();
+		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
+			"cep-starting-snapshot-1.2")));
+		final OperatorStateHandles snapshot = new OperatorStateHandles(
+			(int) ois.readObject(),
+			(StreamStateHandle) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject()
+		);
+		harness.initializeState(snapshot);
+		harness.open();
+
+		harness.processElement(new StreamRecord<>(startEvent2, 5));
+		harness.processElement(new StreamRecord<Event>(middleEvent2, 6));
+		harness.processElement(new StreamRecord<>(endEvent, 7));
+
+		harness.processWatermark(new Watermark(20));
+
+		ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+		// watermark and 3 results
+		assertEquals(4, result.size());
+
+		Object resultObject1 = result.poll();
+		assertTrue(resultObject1 instanceof StreamRecord);
+		StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
+		assertTrue(resultRecord1.getValue() instanceof Map);
+
+		Object resultObject2 = result.poll();
+		assertTrue(resultObject2 instanceof StreamRecord);
+		StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
+		assertTrue(resultRecord2.getValue() instanceof Map);
+
+		Object resultObject3 = result.poll();
+		assertTrue(resultObject3 instanceof StreamRecord);
+		StreamRecord<?> resultRecord3 = (StreamRecord<?>) resultObject3;
+		assertTrue(resultRecord3.getValue() instanceof Map);
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap1 = (Map<String, Event>) resultRecord1.getValue();
+
+		assertEquals(startEvent1, patternMap1.get("start"));
+		assertEquals(middleEvent1, patternMap1.get("middle"));
+		assertEquals(endEvent, patternMap1.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap2 = (Map<String, Event>) resultRecord2.getValue();
+
+		assertEquals(startEvent1, patternMap2.get("start"));
+		assertEquals(middleEvent2, patternMap2.get("middle"));
+		assertEquals(endEvent, patternMap2.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap3 = (Map<String, Event>) resultRecord3.getValue();
+
+		assertEquals(startEvent2, patternMap3.get("start"));
+		assertEquals(middleEvent2, patternMap3.get("middle"));
+		assertEquals(endEvent, patternMap3.get("end"));
+
+		harness.close();
+	}
+
+	@Test
+	public void testSinglePatternAfterMigration() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+
+		// uncomment these lines for regenerating the snapshot on Flink 1.2
+//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+//			new KeyedOneInputStreamOperatorTestHarness<>(
+//				new KeyedCEPPatternOperator<>(
+//					Event.createTypeSerializer(),
+//					false,
+//					keySelector,
+//					IntSerializer.INSTANCE,
+//					new SinglePatternNFAFactory(),
+//					true),
+//				keySelector,
+//				BasicTypeInfo.INT_TYPE_INFO);
+//
+//		harness.setup();
+//		harness.open();
+//		harness.processWatermark(new Watermark(5));
+//		// simulate snapshot/restore with empty element queue but NFA state
+//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
+//		FileOutputStream out = new FileOutputStream(
+//				"src/test/resources/cep-single-pattern-snapshot-1.2");
+//		ObjectOutputStream oos = new ObjectOutputStream(out);
+//		oos.writeObject(snapshot.getOperatorChainIndex());
+//		oos.writeObject(snapshot.getLegacyOperatorState());
+//		oos.writeObject(snapshot.getManagedKeyedState());
+//		oos.writeObject(snapshot.getRawKeyedState());
+//		oos.writeObject(snapshot.getManagedOperatorState());
+//		oos.writeObject(snapshot.getRawOperatorState());
+//		out.close();
+//		harness.close();
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new SinglePatternNFAFactory(),
+					true),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		harness.setup();
+		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
+			"cep-single-pattern-snapshot-1.2")));
+		final OperatorStateHandles snapshot = new OperatorStateHandles(
+			(int) ois.readObject(),
+			(StreamStateHandle) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<KeyGroupsStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject(),
+			(Collection<OperatorStateHandle>) ois.readObject()
+		);
+		harness.initializeState(snapshot);
+		harness.open();
+
+		harness.processElement(new StreamRecord<>(startEvent1, 5));
+
+		harness.processWatermark(new Watermark(20));
+
+		ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+		// watermark and the result
+		assertEquals(2, result.size());
+
+		Object resultObject = result.poll();
+		assertTrue(resultObject instanceof StreamRecord);
+		StreamRecord<?> resultRecord = (StreamRecord<?>) resultObject;
+		assertTrue(resultRecord.getValue() instanceof Map);
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap = (Map<String, Event>) resultRecord.getValue();
+
+		assertEquals(startEvent1, patternMap.get("start"));
+
+		harness.close();
+	}
+
+	private static class SinglePatternNFAFactory implements NFACompiler.NFAFactory<Event> {
+
+		private static final long serialVersionUID = 1173020762472766713L;
+
+		private final boolean handleTimeout;
+
+		private SinglePatternNFAFactory() {
+			this(false);
+		}
+
+		private SinglePatternNFAFactory(boolean handleTimeout) {
+			this.handleTimeout = handleTimeout;
+		}
+
+		@Override
+		public NFA<Event> createNFA() {
+
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
+				.within(Time.milliseconds(10L));
+
+			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+		}
+	}
+
+	private static class NFAFactory implements NFACompiler.NFAFactory<Event> {
+
+		private static final long serialVersionUID = 1173020762472766713L;
+
+		private final boolean handleTimeout;
+
+		private NFAFactory() {
+			this(false);
+		}
+
+		private NFAFactory(boolean handleTimeout) {
+			this.handleTimeout = handleTimeout;
+		}
+
+		@Override
+		public NFA<Event> createNFA() {
+
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
+				.followedBy("middle")
+				.subtype(SubEvent.class)
+				.where(new MiddleFilter())
+				.followedBy("end")
+				.where(new EndFilter())
+				// add a window timeout to test whether timestamps of elements in the
+				// priority queue in CEP operator are correctly checkpointed/restored
+				.within(Time.milliseconds(10L));
+
+			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+		}
+	}
+
+	private static class StartFilter implements FilterFunction<Event> {
+		private static final long serialVersionUID = 5726188262756267490L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("start");
+		}
+	}
+
+	private static class MiddleFilter implements FilterFunction<SubEvent> {
+		private static final long serialVersionUID = 6215754202506583964L;
+
+		@Override
+		public boolean filter(SubEvent value) throws Exception {
+			return value.getVolume() > 5.0;
+		}
+	}
+
+	private static class EndFilter implements FilterFunction<Event> {
+		private static final long serialVersionUID = 7056763917392056548L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("end");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/test/resources/cep-branching-snapshot-1.2
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-branching-snapshot-1.2 b/flink-libraries/flink-cep/src/test/resources/cep-branching-snapshot-1.2
new file mode 100644
index 0000000..47f710e
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-branching-snapshot-1.2 differ

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/test/resources/cep-single-pattern-snapshot-1.2
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-single-pattern-snapshot-1.2 b/flink-libraries/flink-cep/src/test/resources/cep-single-pattern-snapshot-1.2
new file mode 100644
index 0000000..255f46a
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-single-pattern-snapshot-1.2 differ

http://git-wip-us.apache.org/repos/asf/flink/blob/d20fb090/flink-libraries/flink-cep/src/test/resources/cep-starting-snapshot-1.2
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-starting-snapshot-1.2 b/flink-libraries/flink-cep/src/test/resources/cep-starting-snapshot-1.2
new file mode 100644
index 0000000..c41f6c2
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-starting-snapshot-1.2 differ


[14/50] [abbrv] flink git commit: [FLINK-6018] Add tests for KryoSerializer restore with registered types

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/09164cf2/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index faa9314..22bb715 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -18,9 +18,11 @@
 
 package org.apache.flink.runtime.state;
 
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
 import com.google.common.base.Joiner;
 import org.apache.commons.io.output.ByteArrayOutputStream;
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
@@ -34,11 +36,15 @@ import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.FloatSerializer;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+import org.apache.flink.api.java.typeutils.runtime.kryo.JavaSerializer;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
@@ -61,9 +67,11 @@ import org.apache.flink.util.FutureUtil;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -90,6 +98,8 @@ import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
+import org.junit.rules.ExpectedException;
+
 
 /**
  * Generic tests for the partitioned state part of {@link AbstractStateBackend}.
@@ -97,6 +107,9 @@ import static org.mockito.Mockito.verify;
 @SuppressWarnings("serial")
 public abstract class StateBackendTestBase<B extends AbstractStateBackend> extends TestLogger {
 
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
 	protected abstract B getStateBackend() throws Exception;
 
 	protected CheckpointStreamFactory createStreamFactory() throws Exception {
@@ -171,21 +184,478 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 	@Test
 	@SuppressWarnings("unchecked")
+	public void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		// cast because our test serializer is not typed to TestPojo
+		env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) ExceptionThrowingTestSerializer.class);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		// we will be expecting ExpectedKryoTestException to be thrown,
+		// because the ExceptionThrowingTestSerializer should be used
+		int numExceptions = 0;
+
+		backend.setCurrentKey(1);
+
+		try {
+			// backends that eagerly serializes (such as RocksDB) will fail here
+			state.update(new TestPojo("u1", 1));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		try {
+			// backends that lazily serializes (such as memory state backend) will fail here
+			runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		// cast because our test serializer is not typed to TestPojo
+		env.getExecutionConfig()
+				.addDefaultKryoSerializer(TestPojo.class, (Class) ExceptionThrowingTestSerializer.class);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		pojoType.createSerializer(env.getExecutionConfig());
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getOrCreateKeyedState(VoidNamespaceSerializer.INSTANCE, kvId);
+		assertTrue(state instanceof InternalValueState);
+		((InternalValueState) state).setCurrentNamespace(VoidNamespace.INSTANCE);
+
+		// we will be expecting ExpectedKryoTestException to be thrown,
+		// because the ExceptionThrowingTestSerializer should be used
+		int numExceptions = 0;
+
+		backend.setCurrentKey(1);
+
+		try {
+			// backends that eagerly serializes (such as RocksDB) will fail here
+			state.update(new TestPojo("u1", 1));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		try {
+			// backends that lazily serializes (such as memory state backend) will fail here
+			runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+	}
+
+	@Test
+	public void testBackendUsesRegisteredKryoSerializer() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		env.getExecutionConfig()
+				.registerTypeWithKryoSerializer(TestPojo.class, ExceptionThrowingTestSerializer.class);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		// we will be expecting ExpectedKryoTestException to be thrown,
+		// because the ExceptionThrowingTestSerializer should be used
+		int numExceptions = 0;
+
+		backend.setCurrentKey(1);
+
+		try {
+			// backends that eagerly serializes (such as RocksDB) will fail here
+			state.update(new TestPojo("u1", 1));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		try {
+			// backends that lazily serializes (such as memory state backend) will fail here
+			runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, ExceptionThrowingTestSerializer.class);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getOrCreateKeyedState(VoidNamespaceSerializer.INSTANCE, kvId);
+		assertTrue(state instanceof InternalValueState);
+		((InternalValueState) state).setCurrentNamespace(VoidNamespace.INSTANCE);
+
+		// we will be expecting ExpectedKryoTestException to be thrown,
+		// because the ExceptionThrowingTestSerializer should be used
+		int numExceptions = 0;
+
+		backend.setCurrentKey(1);
+
+		try {
+			// backends that eagerly serializes (such as RocksDB) will fail here
+			state.update(new TestPojo("u1", 1));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		try {
+			// backends that lazily serializes (such as memory state backend) will fail here
+			runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		} catch (ExpectedKryoTestException e) {
+			numExceptions++;
+		} catch (Exception e) {
+			if (e.getCause() instanceof ExpectedKryoTestException) {
+				numExceptions++;
+			} else {
+				throw e;
+			}
+		}
+
+		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+	}
+
+
+	/**
+	 * Verify state restore resilience when:
+	 *  - snapshot was taken without any Kryo registrations, specific serializers or default serializers for the state type
+	 *  - restored with the state type registered (no specific serializer)
+	 *
+	 * This test should not fail, because de- / serialization of the state should noth be performed with Kryo's default
+	 * {@link com.esotericsoftware.kryo.serializers.FieldSerializer}.
+	 */
+	@Test
+	public void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		// ============== create snapshot - no Kryo registration or specific / default serializers ==============
+
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.update(new TestPojo("u1", 1));
+
+		backend.setCurrentKey(2);
+		state.update(new TestPojo("u2", 2));
+
+		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+				682375462378L,
+				2,
+				streamFactory,
+				CheckpointOptions.forFullCheckpoint()));
+
+		backend.dispose();
+
+		// ====================================== restore snapshot  ======================================
+
+		env.getExecutionConfig().registerKryoType(TestPojo.class);
+		
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
+
+		snapshot.discardState();
+
+		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		backend.setCurrentKey(1);
+		assertEquals(state.value(), new TestPojo("u1", 1));
+
+		backend.setCurrentKey(2);
+		assertEquals(state.value(), new TestPojo("u2", 2));
+
+		backend.dispose();
+	}
+
+	/**
+	 * Verify state restore resilience when:
+	 *  - snapshot was taken without any Kryo registrations, specific serializers or default serializers for the state type
+	 *  - restored with a default serializer for the state type
+	 *
+	 * <p> The default serializer used on restore is {@link CustomKryoTestSerializer}, which deliberately
+	 * fails only on deserialization. We use the deliberate deserialization failure to acknowledge test success.
+	 *
+	 * @throws Exception expects {@link ExpectedKryoTestException} to be thrown.
+	 */
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+
+		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		// ============== create snapshot - no Kryo registration or specific / default serializers ==============
+
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.update(new TestPojo("u1", 1));
+
+		backend.setCurrentKey(2);
+		state.update(new TestPojo("u2", 2));
+
+		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+				682375462378L,
+				2,
+				streamFactory,
+				CheckpointOptions.forFullCheckpoint()));
+
+		backend.dispose();
+
+		// ========== restore snapshot - should use default serializer (ONLY SERIALIZATION) ==========
+
+		// cast because our test serializer is not typed to TestPojo
+		env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
+
+		snapshot.discardState();
+
+		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
+		// initializeSerializerUnlessSet would not pick up our new config
+		kvId = new ValueStateDescriptor<>("id", pojoType);
+		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		backend.setCurrentKey(1);
+
+		// update to test state backends that eagerly serialize, such as RocksDB
+		state.update(new TestPojo("u1", 11));
+
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(
+				682375462378L,
+				2,
+				streamFactory,
+				CheckpointOptions.forFullCheckpoint()));
+
+		backend.dispose();
+
+		// ========= restore snapshot - should use default serializer (FAIL ON DESERIALIZATION) =========
+
+		// cast because our test serializer is not typed to TestPojo
+		env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
+
+		// on the second restore, since the custom serializer will be used for
+		// deserialization, we expect the deliberate failure to be thrown
+		expectedException.expect(ExpectedKryoTestException.class);
+
+		// state backends that eagerly deserializes (such as the memory state backend) will fail here
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
+
+		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		backend.setCurrentKey(1);
+		// state backends that lazily deserializes (such as RocksDB) will fail here
+		state.value();
+	}
+
+	/**
+	 * Verify state restore resilience when:
+	 *  - snapshot was taken without any Kryo registrations, specific serializers or default serializers for the state type
+	 *  - restored with a specific serializer for the state type
+	 *
+	 * <p> The specific serializer used on restore is {@link CustomKryoTestSerializer}, which deliberately
+	 * fails only on deserialization. We use the deliberate deserialization failure to acknowledge test success.
+	 *
+	 * @throws Exception expects {@link ExpectedKryoTestException} to be thrown.
+	 */
+	@Test
+	public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		Environment env = new DummyEnvironment("test", 1, 0);
+
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+
+		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+
+		// make sure that we are in fact using the KryoSerializer
+		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+
+		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		// ============== create snapshot - no Kryo registration or specific / default serializers ==============
+
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.update(new TestPojo("u1", 1));
+
+		backend.setCurrentKey(2);
+		state.update(new TestPojo("u2", 2));
+
+		KeyGroupsStateHandle snapshot = runSnapshot(backend.snapshot(
+				682375462378L,
+				2,
+				streamFactory,
+				CheckpointOptions.forFullCheckpoint()));
+
+		backend.dispose();
+
+		// ========== restore snapshot - should use specific serializer (ONLY SERIALIZATION) ==========
+
+		env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
+
+		snapshot.discardState();
+
+		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
+		// initializeSerializerUnlessSet would not pick up our new config
+		kvId = new ValueStateDescriptor<>("id", pojoType);
+		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		backend.setCurrentKey(1);
+
+		// update to test state backends that eagerly serialize, such as RocksDB
+		state.update(new TestPojo("u1", 11));
+
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(
+				682375462378L,
+				2,
+				streamFactory,
+				CheckpointOptions.forFullCheckpoint()));
+
+		backend.dispose();
+
+		// ========= restore snapshot - should use specific serializer (FAIL ON DESERIALIZATION) =========
+
+		env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
+
+		// on the second restore, since the custom serializer will be used for
+		// deserialization, we expect the deliberate failure to be thrown
+		expectedException.expect(ExpectedKryoTestException.class);
+
+		// state backends that eagerly deserializes (such as the memory state backend) will fail here
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
+
+		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+
+		backend.setCurrentKey(1);
+		// state backends that lazily deserializes (such as RocksDB) will fail here
+		state.value();
+	}
+
+
+	@Test
+	@SuppressWarnings("unchecked")
 	public void testValueState() throws Exception {
 		CheckpointStreamFactory streamFactory = createStreamFactory();
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
 		TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
-		TypeSerializer<String> valueSerializer = kvId.getSerializer();
 
 		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 		@SuppressWarnings("unchecked")
 		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
+		// this is only available after the backend initialized the serializer
+		TypeSerializer<String> valueSerializer = kvId.getSerializer();
+		
 		// some modifications to the state
 		backend.setCurrentKey(1);
 		assertNull(state.value());
@@ -276,16 +746,17 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		final ValueStateDescriptor<String> kvId =
 			new ValueStateDescriptor<>("id", String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		final TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
 		final TypeSerializer<Integer> namespaceSerializer =
 			IntSerializer.INSTANCE;
-		final TypeSerializer<String> valueSerializer = kvId.getSerializer();
 
 		final ValueState<String> state = backend
 			.getPartitionedState(namespace, IntSerializer.INSTANCE, kvId);
 
+		// this is only available after the backend initialized the serializer
+		final TypeSerializer<String> valueSerializer = kvId.getSerializer();
+
 		@SuppressWarnings("unchecked")
 		final InternalKvState<Integer> kvState = (InternalKvState<Integer>) state;
 
@@ -390,9 +861,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		ValueStateDescriptor<String> desc1 = new ValueStateDescriptor<>("a-string", StringSerializer.INSTANCE);
 		ValueStateDescriptor<Integer> desc2 = new ValueStateDescriptor<>("an-integer", IntSerializer.INSTANCE);
 
-		desc1.initializeSerializerUnlessSet(new ExecutionConfig());
-		desc2.initializeSerializerUnlessSet(new ExecutionConfig());
-
 		ValueState<String> state1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc1);
 		ValueState<Integer> state2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc2);
 
@@ -459,7 +927,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<Long> kvId = new ValueStateDescriptor<>("id", LongSerializer.INSTANCE, 42L);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ValueState<Long> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -499,463 +966,443 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 	@Test
 	@SuppressWarnings("unchecked,rawtypes")
-	public void testListState() {
-		try {
-			CheckpointStreamFactory streamFactory = createStreamFactory();
-			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+	public void testListState() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
-			ListStateDescriptor<String> kvId = new ListStateDescriptor<>("id", String.class);
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+		ListStateDescriptor<String> kvId = new ListStateDescriptor<>("id", String.class);
 
-			TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
-			TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
-			TypeSerializer<String> valueSerializer = kvId.getElementSerializer();
+		TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
+		TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
 
-			ListState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
+		ListState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
-			Joiner joiner = Joiner.on(",");
-			// some modifications to the state
-			backend.setCurrentKey(1);
-			assertEquals(null, state.get());
-			assertEquals(null, getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add("1");
-			backend.setCurrentKey(2);
-			assertEquals(null, state.get());
-			assertEquals(null, getSerializedList(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add("2");
-			backend.setCurrentKey(1);
-			assertEquals("1", joiner.join(state.get()));
-			assertEquals("1", joiner.join(getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		// this is only available after the backend initialized the serializer
+		TypeSerializer<String> valueSerializer = kvId.getElementSerializer();
 
-			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		Joiner joiner = Joiner.on(",");
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals(null, state.get());
+		assertEquals(null, getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add("1");
+		backend.setCurrentKey(2);
+		assertEquals(null, state.get());
+		assertEquals(null, getSerializedList(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add("2");
+		backend.setCurrentKey(1);
+		assertEquals("1", joiner.join(state.get()));
+		assertEquals("1", joiner.join(getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
 
-			// make some more modifications
-			backend.setCurrentKey(1);
-			state.add("u1");
-			backend.setCurrentKey(2);
-			state.add("u2");
-			backend.setCurrentKey(3);
-			state.add("u3");
+		// draw a snapshot
+		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			// draw another snapshot
-			KeyGroupsStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.add("u1");
+		backend.setCurrentKey(2);
+		state.add("u2");
+		backend.setCurrentKey(3);
+		state.add("u3");
 
-			// validate the original state
-			backend.setCurrentKey(1);
-			assertEquals("1,u1", joiner.join(state.get()));
-			assertEquals("1,u1", joiner.join(getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
-			backend.setCurrentKey(2);
-			assertEquals("2,u2", joiner.join(state.get()));
-			assertEquals("2,u2", joiner.join(getSerializedList(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
-			backend.setCurrentKey(3);
-			assertEquals("u3", joiner.join(state.get()));
-			assertEquals("u3", joiner.join(getSerializedList(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		// draw another snapshot
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			backend.dispose();
-			// restore the first snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
-			snapshot1.discardState();
+		// validate the original state
+		backend.setCurrentKey(1);
+		assertEquals("1,u1", joiner.join(state.get()));
+		assertEquals("1,u1", joiner.join(getSerializedList(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		backend.setCurrentKey(2);
+		assertEquals("2,u2", joiner.join(state.get()));
+		assertEquals("2,u2", joiner.join(getSerializedList(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		backend.setCurrentKey(3);
+		assertEquals("u3", joiner.join(state.get()));
+		assertEquals("u3", joiner.join(getSerializedList(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
 
-			ListState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
+		backend.dispose();
+		// restore the first snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+		snapshot1.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("1", joiner.join(restored1.get()));
-			assertEquals("1", joiner.join(getSerializedList(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
-			backend.setCurrentKey(2);
-			assertEquals("2", joiner.join(restored1.get()));
-			assertEquals("2", joiner.join(getSerializedList(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		ListState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
-			backend.dispose();
-			// restore the second snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
-			snapshot2.discardState();
+		backend.setCurrentKey(1);
+		assertEquals("1", joiner.join(restored1.get()));
+		assertEquals("1", joiner.join(getSerializedList(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		backend.setCurrentKey(2);
+		assertEquals("2", joiner.join(restored1.get()));
+		assertEquals("2", joiner.join(getSerializedList(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
 
-			ListState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
+		backend.dispose();
+		// restore the second snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		snapshot2.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("1,u1", joiner.join(restored2.get()));
-			assertEquals("1,u1", joiner.join(getSerializedList(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
-			backend.setCurrentKey(2);
-			assertEquals("2,u2", joiner.join(restored2.get()));
-			assertEquals("2,u2", joiner.join(getSerializedList(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
-			backend.setCurrentKey(3);
-			assertEquals("u3", joiner.join(restored2.get()));
-			assertEquals("u3", joiner.join(getSerializedList(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		ListState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
-			backend.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		backend.setCurrentKey(1);
+		assertEquals("1,u1", joiner.join(restored2.get()));
+		assertEquals("1,u1", joiner.join(getSerializedList(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		backend.setCurrentKey(2);
+		assertEquals("2,u2", joiner.join(restored2.get()));
+		assertEquals("2,u2", joiner.join(getSerializedList(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+		backend.setCurrentKey(3);
+		assertEquals("u3", joiner.join(restored2.get()));
+		assertEquals("u3", joiner.join(getSerializedList(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)));
+
+		backend.dispose();
 	}
 
 	@Test
 	@SuppressWarnings("unchecked")
-	public void testReducingState() {
-		try {
-			CheckpointStreamFactory streamFactory = createStreamFactory();
-			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+	public void testReducingState() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
-			ReducingStateDescriptor<String> kvId = new ReducingStateDescriptor<>("id", new AppendingReduce(), String.class);
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+		ReducingStateDescriptor<String> kvId = new ReducingStateDescriptor<>("id", new AppendingReduce(), String.class);
 
-			TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
-			TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
-			TypeSerializer<String> valueSerializer = kvId.getSerializer();
+		TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
+		TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
 
-			ReducingState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
+		ReducingState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
-			// some modifications to the state
-			backend.setCurrentKey(1);
-			assertEquals(null, state.get());
-			assertNull(getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add("1");
-			backend.setCurrentKey(2);
-			assertEquals(null, state.get());
-			assertNull(getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add("2");
-			backend.setCurrentKey(1);
-			assertEquals("1", state.get());
-			assertEquals("1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		// this is only available after the backend initialized the serializer
+		TypeSerializer<String> valueSerializer = kvId.getSerializer();
 
-			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals(null, state.get());
+		assertNull(getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add("1");
+		backend.setCurrentKey(2);
+		assertEquals(null, state.get());
+		assertNull(getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add("2");
+		backend.setCurrentKey(1);
+		assertEquals("1", state.get());
+		assertEquals("1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			// make some more modifications
-			backend.setCurrentKey(1);
-			state.add("u1");
-			backend.setCurrentKey(2);
-			state.add("u2");
-			backend.setCurrentKey(3);
-			state.add("u3");
+		// draw a snapshot
+		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			// draw another snapshot
-			KeyGroupsStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.add("u1");
+		backend.setCurrentKey(2);
+		state.add("u2");
+		backend.setCurrentKey(3);
+		state.add("u3");
 
-			// validate the original state
-			backend.setCurrentKey(1);
-			assertEquals("1,u1", state.get());
-			assertEquals("1,u1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("2,u2", state.get());
-			assertEquals("2,u2", getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(3);
-			assertEquals("u3", state.get());
-			assertEquals("u3", getSerializedValue(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		// draw another snapshot
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			backend.dispose();
-			// restore the first snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
-			snapshot1.discardState();
+		// validate the original state
+		backend.setCurrentKey(1);
+		assertEquals("1,u1", state.get());
+		assertEquals("1,u1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("2,u2", state.get());
+		assertEquals("2,u2", getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(3);
+		assertEquals("u3", state.get());
+		assertEquals("u3", getSerializedValue(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			ReducingState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
+		backend.dispose();
+		// restore the first snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+		snapshot1.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("1", restored1.get());
-			assertEquals("1", getSerializedValue(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("2", restored1.get());
-			assertEquals("2", getSerializedValue(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		ReducingState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
-			backend.dispose();
-			// restore the second snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
-			snapshot2.discardState();
+		backend.setCurrentKey(1);
+		assertEquals("1", restored1.get());
+		assertEquals("1", getSerializedValue(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("2", restored1.get());
+		assertEquals("2", getSerializedValue(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			ReducingState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
+		backend.dispose();
+		// restore the second snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		snapshot2.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("1,u1", restored2.get());
-			assertEquals("1,u1", getSerializedValue(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("2,u2", restored2.get());
-			assertEquals("2,u2", getSerializedValue(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(3);
-			assertEquals("u3", restored2.get());
-			assertEquals("u3", getSerializedValue(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		ReducingState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
-			backend.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		backend.setCurrentKey(1);
+		assertEquals("1,u1", restored2.get());
+		assertEquals("1,u1", getSerializedValue(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("2,u2", restored2.get());
+		assertEquals("2,u2", getSerializedValue(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(3);
+		assertEquals("u3", restored2.get());
+		assertEquals("u3", getSerializedValue(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+
+		backend.dispose();
 	}
 
 	@Test
 	@SuppressWarnings("unchecked,rawtypes")
-	public void testFoldingState() {
-		try {
-			CheckpointStreamFactory streamFactory = createStreamFactory();
-			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+	public void testFoldingState() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
-			FoldingStateDescriptor<Integer, String> kvId = new FoldingStateDescriptor<>("id",
-					"Fold-Initial:",
-					new AppendingFold(),
-					String.class);
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+		FoldingStateDescriptor<Integer, String> kvId = new FoldingStateDescriptor<>("id",
+				"Fold-Initial:",
+				new AppendingFold(),
+				String.class);
 
-			TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
-			TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
-			TypeSerializer<String> valueSerializer = kvId.getSerializer();
+		TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
+		TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
 
-			FoldingState<Integer, String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
+		FoldingState<Integer, String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
-			// some modifications to the state
-			backend.setCurrentKey(1);
-			assertEquals(null, state.get());
-			assertEquals(null, getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add(1);
-			backend.setCurrentKey(2);
-			assertEquals(null, state.get());
-			assertEquals(null, getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			state.add(2);
-			backend.setCurrentKey(1);
-			assertEquals("Fold-Initial:,1", state.get());
-			assertEquals("Fold-Initial:,1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		// this is only available after the backend initialized the serializer
+		TypeSerializer<String> valueSerializer = kvId.getSerializer();
 
-			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals(null, state.get());
+		assertEquals(null, getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add(1);
+		backend.setCurrentKey(2);
+		assertEquals(null, state.get());
+		assertEquals(null, getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		state.add(2);
+		backend.setCurrentKey(1);
+		assertEquals("Fold-Initial:,1", state.get());
+		assertEquals("Fold-Initial:,1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			// make some more modifications
-			backend.setCurrentKey(1);
-			state.clear();
-			state.add(101);
-			backend.setCurrentKey(2);
-			state.add(102);
-			backend.setCurrentKey(3);
-			state.add(103);
+		// draw a snapshot
+		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			// draw another snapshot
-			KeyGroupsStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.clear();
+		state.add(101);
+		backend.setCurrentKey(2);
+		state.add(102);
+		backend.setCurrentKey(3);
+		state.add(103);
 
-			// validate the original state
-			backend.setCurrentKey(1);
-			assertEquals("Fold-Initial:,101", state.get());
-			assertEquals("Fold-Initial:,101", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("Fold-Initial:,2,102", state.get());
-			assertEquals("Fold-Initial:,2,102", getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(3);
-			assertEquals("Fold-Initial:,103", state.get());
-			assertEquals("Fold-Initial:,103", getSerializedValue(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		// draw another snapshot
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			backend.dispose();
-			// restore the first snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
-			snapshot1.discardState();
+		// validate the original state
+		backend.setCurrentKey(1);
+		assertEquals("Fold-Initial:,101", state.get());
+		assertEquals("Fold-Initial:,101", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("Fold-Initial:,2,102", state.get());
+		assertEquals("Fold-Initial:,2,102", getSerializedValue(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(3);
+		assertEquals("Fold-Initial:,103", state.get());
+		assertEquals("Fold-Initial:,103", getSerializedValue(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			FoldingState<Integer, String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
+		backend.dispose();
+		// restore the first snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+		snapshot1.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("Fold-Initial:,1", restored1.get());
-			assertEquals("Fold-Initial:,1", getSerializedValue(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("Fold-Initial:,2", restored1.get());
-			assertEquals("Fold-Initial:,2", getSerializedValue(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		FoldingState<Integer, String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
-			backend.dispose();
-			// restore the second snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
-			snapshot1.discardState();
+		backend.setCurrentKey(1);
+		assertEquals("Fold-Initial:,1", restored1.get());
+		assertEquals("Fold-Initial:,1", getSerializedValue(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("Fold-Initial:,2", restored1.get());
+		assertEquals("Fold-Initial:,2", getSerializedValue(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
 
-			@SuppressWarnings("unchecked")
-			FoldingState<Integer, String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
+		backend.dispose();
+		// restore the second snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		snapshot1.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("Fold-Initial:,101", restored2.get());
-			assertEquals("Fold-Initial:,101", getSerializedValue(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("Fold-Initial:,2,102", restored2.get());
-			assertEquals("Fold-Initial:,2,102", getSerializedValue(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
-			backend.setCurrentKey(3);
-			assertEquals("Fold-Initial:,103", restored2.get());
-			assertEquals("Fold-Initial:,103", getSerializedValue(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		@SuppressWarnings("unchecked")
+		FoldingState<Integer, String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
-			backend.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		backend.setCurrentKey(1);
+		assertEquals("Fold-Initial:,101", restored2.get());
+		assertEquals("Fold-Initial:,101", getSerializedValue(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("Fold-Initial:,2,102", restored2.get());
+		assertEquals("Fold-Initial:,2,102", getSerializedValue(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+		backend.setCurrentKey(3);
+		assertEquals("Fold-Initial:,103", restored2.get());
+		assertEquals("Fold-Initial:,103", getSerializedValue(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer));
+
+		backend.dispose();
 	}
 
 	@Test
 	@SuppressWarnings("unchecked,rawtypes")
-	public void testMapState() {
-		try {
-			CheckpointStreamFactory streamFactory = createStreamFactory();
-			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+	public void testMapState() throws Exception {
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
-			MapStateDescriptor<Integer, String> kvId = new MapStateDescriptor<>("id", Integer.class, String.class);
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+		MapStateDescriptor<Integer, String> kvId = new MapStateDescriptor<>("id", Integer.class, String.class);
 
-			TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
-			TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
-			TypeSerializer<Integer> userKeySerializer = kvId.getKeySerializer();
-			TypeSerializer<String> userValueSerializer = kvId.getValueSerializer();
+		TypeSerializer<Integer> keySerializer = IntSerializer.INSTANCE;
+		TypeSerializer<VoidNamespace> namespaceSerializer = VoidNamespaceSerializer.INSTANCE;
 
-			MapState<Integer, String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
+		MapState<Integer, String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
-			// some modifications to the state
-			backend.setCurrentKey(1);
-			assertEquals(null, state.get(1));
-			assertEquals(null, getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			state.put(1, "1");
-			backend.setCurrentKey(2);
-			assertEquals(null, state.get(2));
-			assertEquals(null, getSerializedMap(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			state.put(2, "2");
-			backend.setCurrentKey(1);
-			assertTrue(state.contains(1));
-			assertEquals("1", state.get(1));
-			assertEquals(new HashMap<Integer, String>() {{ put (1, "1"); }},
-					getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		// these are only available after the backend initialized the serializer
+		TypeSerializer<Integer> userKeySerializer = kvId.getKeySerializer();
+		TypeSerializer<String> userValueSerializer = kvId.getValueSerializer();
 
-			// draw a snapshot
-			KeyGroupsStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals(null, state.get(1));
+		assertEquals(null, getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		state.put(1, "1");
+		backend.setCurrentKey(2);
+		assertEquals(null, state.get(2));
+		assertEquals(null, getSerializedMap(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		state.put(2, "2");
+		backend.setCurrentKey(1);
+		assertTrue(state.contains(1));
+		assertEquals("1", state.get(1));
+		assertEquals(new HashMap<Integer, String>() {{ put (1, "1"); }},
+				getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
 
-			// make some more modifications
-			backend.setCurrentKey(1);
-			state.put(1, "101");
-			backend.setCurrentKey(2);
-			state.put(102, "102");
-			backend.setCurrentKey(3);
-			state.put(103, "103");
-			state.putAll(new HashMap<Integer, String>() {{ put(1031, "1031"); put(1032, "1032"); }});
+		// draw a snapshot
+		KeyGroupsStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			// draw another snapshot
-			KeyGroupsStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.put(1, "101");
+		backend.setCurrentKey(2);
+		state.put(102, "102");
+		backend.setCurrentKey(3);
+		state.put(103, "103");
+		state.putAll(new HashMap<Integer, String>() {{ put(1031, "1031"); put(1032, "1032"); }});
 
-			// validate the original state
-			backend.setCurrentKey(1);
-			assertEquals("101", state.get(1));
-			assertEquals(new HashMap<Integer, String>() {{ put(1, "101"); }},
-					getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("102", state.get(102));
-			assertEquals(new HashMap<Integer, String>() {{ put(2, "2"); put(102, "102"); }},
-					getSerializedMap(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			backend.setCurrentKey(3);
-			assertTrue(state.contains(103));
-			assertEquals("103", state.get(103));
-			assertEquals(new HashMap<Integer, String>() {{ put(103, "103"); put(1031, "1031"); put(1032, "1032"); }},
-					getSerializedMap(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-
-			List<Integer> keys = new ArrayList<>();
-			for (Integer key : state.keys()) {
-				keys.add(key);
-			}
-			List<Integer> expectedKeys = new ArrayList<Integer>() {{ add(103); add(1031); add(1032); }};
-			assertEquals(keys.size(), expectedKeys.size());
-			keys.removeAll(expectedKeys);
-			assertTrue(keys.isEmpty());
-
-			List<String> values = new ArrayList<>();
-			for (String value : state.values()) {
-				values.add(value);
-			}
-			List<String> expectedValues = new ArrayList<String>() {{ add("103"); add("1031"); add("1032"); }};
-			assertEquals(values.size(), expectedValues.size());
-			values.removeAll(expectedValues);
-			assertTrue(values.isEmpty());
+		// draw another snapshot
+		KeyGroupsStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forFullCheckpoint()));
 
-			// make some more modifications
-			backend.setCurrentKey(1);
-			state.clear();
-			backend.setCurrentKey(2);
-			state.remove(102);
-			backend.setCurrentKey(3);
-			final String updateSuffix = "_updated";
-			Iterator<Map.Entry<Integer, String>> iterator = state.iterator();
-			while (iterator.hasNext()) {
-				Map.Entry<Integer, String> entry = iterator.next();
-				if (entry.getValue().length() != 4) {
-					iterator.remove();
-				} else {
-					entry.setValue(entry.getValue() + updateSuffix);
-				}
-			}
+		// validate the original state
+		backend.setCurrentKey(1);
+		assertEquals("101", state.get(1));
+		assertEquals(new HashMap<Integer, String>() {{ put(1, "101"); }},
+				getSerializedMap(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("102", state.get(102));
+		assertEquals(new HashMap<Integer, String>() {{ put(2, "2"); put(102, "102"); }},
+				getSerializedMap(kvState, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		backend.setCurrentKey(3);
+		assertTrue(state.contains(103));
+		assertEquals("103", state.get(103));
+		assertEquals(new HashMap<Integer, String>() {{ put(103, "103"); put(1031, "1031"); put(1032, "1032"); }},
+				getSerializedMap(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+
+		List<Integer> keys = new ArrayList<>();
+		for (Integer key : state.keys()) {
+			keys.add(key);
+		}
+		List<Integer> expectedKeys = new ArrayList<Integer>() {{ add(103); add(1031); add(1032); }};
+		assertEquals(keys.size(), expectedKeys.size());
+		keys.removeAll(expectedKeys);
+		assertTrue(keys.isEmpty());
+
+		List<String> values = new ArrayList<>();
+		for (String value : state.values()) {
+			values.add(value);
+		}
+		List<String> expectedValues = new ArrayList<String>() {{ add("103"); add("1031"); add("1032"); }};
+		assertEquals(values.size(), expectedValues.size());
+		values.removeAll(expectedValues);
+		assertTrue(values.isEmpty());
 
-			// validate the state
-			backend.setCurrentKey(1);
-			backend.setCurrentKey(2);
-			assertFalse(state.contains(102));
-			backend.setCurrentKey(3);
-			for (Map.Entry<Integer, String> entry : state.entries()) {
-				assertEquals(4 + updateSuffix.length(), entry.getValue().length());
-				assertTrue(entry.getValue().endsWith(updateSuffix));
+		// make some more modifications
+		backend.setCurrentKey(1);
+		state.clear();
+		backend.setCurrentKey(2);
+		state.remove(102);
+		backend.setCurrentKey(3);
+		final String updateSuffix = "_updated";
+		Iterator<Map.Entry<Integer, String>> iterator = state.iterator();
+		while (iterator.hasNext()) {
+			Map.Entry<Integer, String> entry = iterator.next();
+			if (entry.getValue().length() != 4) {
+				iterator.remove();
+			} else {
+				entry.setValue(entry.getValue() + updateSuffix);
 			}
+		}
 
-			backend.dispose();
-			// restore the first snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
-			snapshot1.discardState();
+		// validate the state
+		backend.setCurrentKey(1);
+		backend.setCurrentKey(2);
+		assertFalse(state.contains(102));
+		backend.setCurrentKey(3);
+		for (Map.Entry<Integer, String> entry : state.entries()) {
+			assertEquals(4 + updateSuffix.length(), entry.getValue().length());
+			assertTrue(entry.getValue().endsWith(updateSuffix));
+		}
 
-			MapState<Integer, String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
+		backend.dispose();
+		// restore the first snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+		snapshot1.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("1", restored1.get(1));
-			assertEquals(new HashMap<Integer, String>() {{ put (1, "1"); }},
-					getSerializedMap(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("2", restored1.get(2));
-			assertEquals(new HashMap<Integer, String>() {{ put (2, "2"); }},
-					getSerializedMap(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		MapState<Integer, String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
-			backend.dispose();
-			// restore the second snapshot and validate it
-			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
-			snapshot2.discardState();
+		backend.setCurrentKey(1);
+		assertEquals("1", restored1.get(1));
+		assertEquals(new HashMap<Integer, String>() {{ put (1, "1"); }},
+				getSerializedMap(restoredKvState1, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("2", restored1.get(2));
+		assertEquals(new HashMap<Integer, String>() {{ put (2, "2"); }},
+				getSerializedMap(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
 
-			@SuppressWarnings("unchecked")
-			MapState<Integer, String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
-			@SuppressWarnings("unchecked")
-			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
+		backend.dispose();
+		// restore the second snapshot and validate it
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		snapshot2.discardState();
 
-			backend.setCurrentKey(1);
-			assertEquals("101", restored2.get(1));
-			assertEquals(new HashMap<Integer, String>() {{ put (1, "101"); }},
-					getSerializedMap(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			backend.setCurrentKey(2);
-			assertEquals("102", restored2.get(102));
-			assertEquals(new HashMap<Integer, String>() {{ put(2, "2"); put (102, "102"); }},
-					getSerializedMap(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
-			backend.setCurrentKey(3);
-			assertEquals("103", restored2.get(103));
-			assertEquals(new HashMap<Integer, String>() {{ put(103, "103"); put(1031, "1031"); put(1032, "1032"); }},
-					getSerializedMap(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		@SuppressWarnings("unchecked")
+		MapState<Integer, String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+		@SuppressWarnings("unchecked")
+		InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
-			backend.dispose();
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		backend.setCurrentKey(1);
+		assertEquals("101", restored2.get(1));
+		assertEquals(new HashMap<Integer, String>() {{ put (1, "101"); }},
+				getSerializedMap(restoredKvState2, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		backend.setCurrentKey(2);
+		assertEquals("102", restored2.get(102));
+		assertEquals(new HashMap<Integer, String>() {{ put(2, "2"); put (102, "102"); }},
+				getSerializedMap(restoredKvState2, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
+		backend.setCurrentKey(3);
+		assertEquals("103", restored2.get(103));
+		assertEquals(new HashMap<Integer, String>() {{ put(103, "103"); put(1031, "1031"); put(1032, "1032"); }},
+				getSerializedMap(restoredKvState2, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, userKeySerializer, userValueSerializer));
 
+		backend.dispose();
 	}
 
 	/**
@@ -966,7 +1413,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class, null);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -991,7 +1437,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class, "Hello");
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -1015,7 +1460,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ReducingStateDescriptor<String> kvId = new ReducingStateDescriptor<>("id", new AppendingReduce(), String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ReducingState<String> state = backend.getPartitionedState(
 				VoidNamespace.INSTANCE,
@@ -1043,8 +1487,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		FoldingStateDescriptor<Integer, String> kvId =
 				new FoldingStateDescriptor<>("id", "Fold-Initial:", new AppendingFold(), String.class);
 
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
-
 		FoldingState<Integer, String> state = backend.getPartitionedState(
 				VoidNamespace.INSTANCE,
 				VoidNamespaceSerializer.INSTANCE, kvId);
@@ -1071,7 +1513,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ListStateDescriptor<String> kvId = new ListStateDescriptor<>("id", String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ListState<String> state = backend.getPartitionedState(
 				VoidNamespace.INSTANCE,
@@ -1098,7 +1539,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		MapStateDescriptor<String, String> kvId = new MapStateDescriptor<>("id", String.class, String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		MapState<String, String> state = backend.getPartitionedState(
 				VoidNamespace.INSTANCE,
@@ -1142,7 +1582,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				new DummyEnvironment("test", 1, 0));
 
 		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class);
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -1224,7 +1663,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 			ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>("id", String.class);
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -1430,7 +1868,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<IntValue> kvId = new ValueStateDescriptor<>("id", IntValue.class, new IntValue(-1));
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		ValueState<IntValue> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
@@ -1458,7 +1895,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<IntValue> kvId = new ValueStateDescriptor<>("id", IntValue.class, new IntValue(-1));
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
 		try {
 			backend.getPartitionedState(null, VoidNamespaceSerializer.INSTANCE, kvId);
@@ -1501,7 +1937,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 					Integer.class,
 					-1);
 			desc.setQueryable("my-query");
-			desc.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			ValueState<Integer> state = backend.getPartitionedState(
 					VoidNamespace.INSTANCE,
@@ -1524,7 +1959,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			// ListState
 			ListStateDescriptor<Integer> desc = new ListStateDescriptor<>("list-state", Integer.class);
 			desc.setQueryable("my-query");
-			desc.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			ListState<Integer> state = backend.getPartitionedState(
 					VoidNamespace.INSTANCE,
@@ -1552,7 +1986,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				}
 			}, Integer.class);
 			desc.setQueryable("my-query");
-			desc.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			ReducingState<Integer> state = backend.getPartitionedState(
 					VoidNamespace.INSTANCE,
@@ -1580,7 +2013,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				}
 			}, Integer.class);
 			desc.setQueryable("my-query");
-			desc.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			FoldingState<Integer, Integer> state = backend.getPartitionedState(
 					VoidNamespace.INSTANCE,
@@ -1602,7 +2034,6 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			// MapState
 			MapStateDescriptor<Integer, String> desc = new MapStateDescriptor<>("map-state", Integer.class, String.class);
 			desc.setQueryable("my-query");
-			desc.initializeSerializerUnlessSet(new ExecutionConfig());
 
 			MapState<Integer, String> state = backend.getPartitionedState(
 					VoidNamespace.INSTANCE,
@@ -1935,4 +2366,107 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			return KvStateRequestSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
 		}
 	}
+
+	private KeyGroupsStateHandle runSnapshot(RunnableFuture<KeyGroupsStateHandle> snapshotRunnableFuture) throws Exception {
+		if(!snapshotRunnableFuture.isDone()) {
+			Thread runner = new Thread(snapshotRunnableFuture);
+			runner.start();
+		}
+		return snapshotRunnableFuture.get();
+	}
+
+	private static class TestPojo implements Serializable {
+		private String strField;
+		private Integer intField;
+
+		public TestPojo() {}
+
+		public TestPojo(String strField, Integer intField) {
+			this.strField = strField;
+			this.intField = intField;
+		}
+
+		public String getStrField() {
+			return strField;
+		}
+
+		public void setStrField(String strField) {
+			this.strField = strField;
+		}
+
+		public Integer getIntField() {
+			return intField;
+		}
+
+		public void setIntField(Integer intField) {
+			this.intField = intField;
+		}
+
+		@Override
+		public String toString() {
+			return "TestPojo{" +
+					"strField='" + strField + '\'' +
+					", intField=" + intField +
+					'}';
+		}
+
+		@Override
+		public boolean equals(Object o) {
+			if (this == o) return true;
+			if (o == null || getClass() != o.getClass()) return false;
+
+			TestPojo testPojo = (TestPojo) o;
+
+			if (!strField.equals(testPojo.strField)) return false;
+			return intField.equals(testPojo.intField);
+		}
+
+		@Override
+		public int hashCode() {
+			int result = strField.hashCode();
+			result = 31 * result + intField.hashCode();
+			return result;
+		}
+	}
+
+	/**
+	 * We throw this in our {@link ExceptionThrowingTestSerializer}.
+	 */
+	private static class ExpectedKryoTestException extends RuntimeException {}
+
+	/**
+	 * Kryo {@code Serializer} that throws an expected exception. We use this to ensure
+	 * that the state backend correctly uses a specified Kryo serializer.
+	 */
+	public static class ExceptionThrowingTestSerializer extends JavaSerializer {
+		@Override
+		public void write(Kryo kryo, Output output, Object object) {
+			throw new ExpectedKryoTestException();
+		}
+
+		@Override
+		public Object read(Kryo kryo, Input input, Class type) {
+			throw new ExpectedKryoTestException();
+		}
+	}
+
+	/**
+	 * Our custom version of {@link JavaSerializer} for checking whether restore with a registered
+	 * serializer works when no serializer was previously registered.
+	 *
+	 * <p>This {@code Serializer} can only be used for writing, not for reading. With this we
+	 * verify that state that was serialized without a registered {@code Serializer} is in fact
+	 * not restored with a {@code Serializer} that was later registered.
+	 */
+	public static class CustomKryoTestSerializer extends JavaSerializer {
+		@Override
+		public void write(Kryo kryo, Output output, Object object) {
+			super.write(kryo, output, object);
+		}
+
+		@Override
+		public Object read(Kryo kryo, Input input, Class type) {
+			throw new ExpectedKryoTestException();
+		}
+	}
 }