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

[1/9] flink git commit: [hotfix] [streaming api] Non-merging triggers throw UnsupportedOperationException instead of RuntimeException

Repository: flink
Updated Branches:
  refs/heads/master 392b2e9a0 -> 09380e492


[hotfix] [streaming api] Non-merging triggers throw UnsupportedOperationException instead of RuntimeException


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

Branch: refs/heads/master
Commit: 2f1c4740cebcdd66aad119214501bb51a1031d3d
Parents: 254a700
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Jan 19 18:57:16 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:22:21 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/streaming/api/windowing/triggers/Trigger.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2f1c4740/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
index a0209aa..11a0d6d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
@@ -105,7 +105,7 @@ public abstract class Trigger<T, W extends Window> implements Serializable {
 	 * @param ctx A context object that can be used to register timer callbacks and access state.
 	 */
 	public void onMerge(W window, OnMergeContext ctx) throws Exception {
-		throw new RuntimeException("This trigger does not support merging.");
+		throw new UnsupportedOperationException("This trigger does not support merging.");
 	}
 
 	/**


[3/9] flink git commit: [hotfix] Cleanups of the AbstractKeyedStateBackend

Posted by se...@apache.org.
[hotfix] Cleanups of the AbstractKeyedStateBackend


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

Branch: refs/heads/master
Commit: 51c02eee7811116f9879d28e79967685a6a037ac
Parents: 392b2e9
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jan 13 14:50:30 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:22:21 2017 +0100

----------------------------------------------------------------------
 .../state/AbstractKeyedStateBackend.java        | 116 +++++++++++++------
 1 file changed, 78 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/51c02eee/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 2daf896..cab2b4f 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
@@ -291,48 +291,88 @@ public abstract class AbstractKeyedStateBackend<K>
 	}
 
 	@Override
-	@SuppressWarnings("unchecked,rawtypes")
-	public <N, S extends MergingState<?, ?>> void mergePartitionedStates(final N target, Collection<N> sources, final TypeSerializer<N> namespaceSerializer, final StateDescriptor<S, ?> stateDescriptor) throws Exception {
+	public <N, S extends MergingState<?, ?>> void mergePartitionedStates(
+			final N target,
+			Collection<N> sources,
+			final TypeSerializer<N> namespaceSerializer,
+			final StateDescriptor<S, ?> stateDescriptor) throws Exception {
+
 		if (stateDescriptor instanceof ReducingStateDescriptor) {
-			ReducingStateDescriptor reducingStateDescriptor = (ReducingStateDescriptor) stateDescriptor;
-			ReduceFunction reduceFn = reducingStateDescriptor.getReduceFunction();
-			ReducingState state = (ReducingState) getPartitionedState(target, namespaceSerializer, stateDescriptor);
-			KvState kvState = (KvState) state;
-			Object result = null;
-			for (N source: sources) {
-				kvState.setCurrentNamespace(source);
-				Object sourceValue = state.get();
-				if (result == null) {
-					result = state.get();
-				} else if (sourceValue != null) {
-					result = reduceFn.reduce(result, sourceValue);
-				}
-				state.clear();
-			}
-			kvState.setCurrentNamespace(target);
-			if (result != null) {
-				state.add(result);
+			mergeReducingState((ReducingStateDescriptor<?>) stateDescriptor, namespaceSerializer, target,sources);
+		}
+		else if (stateDescriptor instanceof ListStateDescriptor) {
+			mergeListState((ListStateDescriptor<?>) stateDescriptor, namespaceSerializer, target,sources);
+		}
+		else {
+			throw new IllegalArgumentException("Cannot merge states for " + stateDescriptor);
+		}
+	}
+
+	private <N, T> void mergeReducingState(
+			final ReducingStateDescriptor<?> stateDescriptor,
+			final TypeSerializer<N> namespaceSerializer,
+			final N target,
+			final Collection<N> sources) throws Exception {
+
+		@SuppressWarnings("unchecked")
+		final ReducingStateDescriptor<T> reducingStateDescriptor = (ReducingStateDescriptor<T>) stateDescriptor;
+
+		@SuppressWarnings("unchecked")
+		final ReducingState<T> state = (ReducingState<T>) getPartitionedState(target, namespaceSerializer, stateDescriptor);
+
+		@SuppressWarnings("unchecked")
+		final KvState<N> kvState = (KvState<N>) state;
+
+		final ReduceFunction<T> reduceFn = reducingStateDescriptor.getReduceFunction();
+
+		T result = null;
+		for (N source: sources) {
+			kvState.setCurrentNamespace(source);
+			T sourceValue = state.get();
+			if (result == null) {
+				result = state.get();
+			} else if (sourceValue != null) {
+				result = reduceFn.reduce(result, sourceValue);
 			}
-		} else if (stateDescriptor instanceof ListStateDescriptor) {
-			ListState<Object> state = (ListState) getPartitionedState(target, namespaceSerializer, stateDescriptor);
-			KvState kvState = (KvState) state;
-			List<Object> result = new ArrayList<>();
-			for (N source: sources) {
-				kvState.setCurrentNamespace(source);
-				Iterable<Object> sourceValue = state.get();
-				if (sourceValue != null) {
-					for (Object o : sourceValue) {
-						result.add(o);
-					}
+			state.clear();
+		}
+
+		// write result to the target
+		kvState.setCurrentNamespace(target);
+		if (result != null) {
+			state.add(result);
+		}
+	}
+
+	private <N, T> void mergeListState(
+			final ListStateDescriptor<?> listStateDescriptor,
+			final TypeSerializer<N> namespaceSerializer,
+			final N target,
+			final Collection<N> sources) throws Exception {
+
+		@SuppressWarnings("unchecked")
+		final ListState<T> state = (ListState<T>) getPartitionedState(target, namespaceSerializer, listStateDescriptor);
+
+		@SuppressWarnings("unchecked")
+		final KvState<N> kvState = (KvState<N>) state;
+
+		// merge the sources
+		final List<T> result = new ArrayList<>();
+		for (N source: sources) {
+			kvState.setCurrentNamespace(source);
+			Iterable<T> sourceValue = state.get();
+			if (sourceValue != null) {
+				for (T o : sourceValue) {
+					result.add(o);
 				}
-				state.clear();
-			}
-			kvState.setCurrentNamespace(target);
-			for (Object o : result) {
-				state.add(o);
 			}
-		} else {
-			throw new RuntimeException("Cannot merge states for " + stateDescriptor);
+			state.clear();
+		}
+
+		// write to the target
+		kvState.setCurrentNamespace(target);
+		for (T o : result) {
+			state.add(o);
 		}
 	}
 


[6/9] flink git commit: [FLINK-5590] [runtime] Add proper internal state hierarchy

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
index 9552325..f0eb53e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.ByteArrayOutputStream;
@@ -39,8 +40,8 @@ import java.util.Map;
  * @param <V> The type of the value.
  */
 public class HeapListState<K, N, V>
-		extends AbstractHeapState<K, N, ArrayList<V>, ListState<V>, ListStateDescriptor<V>>
-		implements ListState<V> {
+		extends AbstractHeapMergingState<K, N, V, Iterable<V>, ArrayList<V>, ListState<V>, ListStateDescriptor<V>>
+		implements InternalListState<N, V> {
 
 	/**
 	 * Creates a new key/value state for the given hash map of key/value pairs.
@@ -59,6 +60,10 @@ public class HeapListState<K, N, V>
 		super(backend, stateDesc, stateTable, keySerializer, namespaceSerializer);
 	}
 
+	// ------------------------------------------------------------------------
+	//  state access
+	// ------------------------------------------------------------------------
+
 	@Override
 	public Iterable<V> get() {
 		Preconditions.checkState(currentNamespace != null, "No namespace set.");
@@ -154,4 +159,14 @@ public class HeapListState<K, N, V>
 
 		return baos.toByteArray();
 	}
+
+	// ------------------------------------------------------------------------
+	//  state merging
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected ArrayList<V> mergeState(ArrayList<V> a, ArrayList<V> b) {
+		a.addAll(b);
+		return a;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
index 37aa812..7804cb4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -37,8 +38,8 @@ import java.util.Map;
  * @param <V> The type of the value.
  */
 public class HeapReducingState<K, N, V>
-		extends AbstractHeapState<K, N, V, ReducingState<V>, ReducingStateDescriptor<V>>
-		implements ReducingState<V> {
+		extends AbstractHeapMergingState<K, N, V, V, V, ReducingState<V>, ReducingStateDescriptor<V>>
+		implements InternalReducingState<N, V> {
 
 	private final ReduceFunction<V> reduceFunction;
 
@@ -56,10 +57,15 @@ public class HeapReducingState<K, N, V>
 			StateTable<K, N, V> stateTable,
 			TypeSerializer<K> keySerializer,
 			TypeSerializer<N> namespaceSerializer) {
+
 		super(backend, stateDesc, stateTable, keySerializer, namespaceSerializer);
 		this.reduceFunction = stateDesc.getReduceFunction();
 	}
 
+	// ------------------------------------------------------------------------
+	//  state access
+	// ------------------------------------------------------------------------
+
 	@Override
 	public V get() {
 		Preconditions.checkState(currentNamespace != null, "No namespace set.");
@@ -111,13 +117,22 @@ public class HeapReducingState<K, N, V>
 		if (currentValue == null) {
 			// we're good, just added the new value
 		} else {
-			V reducedValue = null;
+			V reducedValue;
 			try {
 				reducedValue = reduceFunction.reduce(currentValue, value);
 			} catch (Exception e) {
-				throw new RuntimeException("Could not add value to reducing state.", e);
+				throw new IOException("Exception while applying ReduceFunction in reducing state", e);
 			}
 			keyedMap.put(backend.<K>getCurrentKey(), reducedValue);
 		}
 	}
+
+	// ------------------------------------------------------------------------
+	//  state merging
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected V mergeState(V a, V b) throws Exception {
+		return reduceFunction.reduce(a, b);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java
index cccaacb..9e042fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.util.Preconditions;
 
 import java.util.Map;
@@ -36,7 +37,7 @@ import java.util.Map;
  */
 public class HeapValueState<K, N, V>
 		extends AbstractHeapState<K, N, V, ValueState<V>, ValueStateDescriptor<V>>
-		implements ValueState<V> {
+		implements InternalValueState<N, V> {
 
 	/**
 	 * Creates a new key/value state for the given hash map of key/value pairs.

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
index 797150a..9d7232e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.runtime.state.heap;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.KeyGroupRange;
@@ -78,4 +79,21 @@ public class StateTable<K, N, ST> {
 	public List<Map<N, Map<K, ST>>> getState() {
 		return state;
 	}
+
+	// ------------------------------------------------------------------------
+	//  for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isEmpty() {
+		for (Map<N, Map<K, ST>> map : state) {
+			if (map != null) {
+				if (!map.isEmpty()) {
+					return false;
+				}
+			}
+		}
+
+		return true;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAppendingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAppendingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAppendingState.java
new file mode 100644
index 0000000..ae9f457
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAppendingState.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.runtime.state.internal;
+
+import org.apache.flink.api.common.state.AppendingState;
+
+/**
+ * The peer to the {@link AppendingState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N>   The type of the namespace
+ * @param <IN>  The type of elements added to the state
+ * @param <OUT> The type of the 
+ */
+public interface InternalAppendingState<N, IN, OUT> extends InternalKvState<N>, AppendingState<IN, OUT> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalFoldingState.java
new file mode 100644
index 0000000..eb58ce5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalFoldingState.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.runtime.state.internal;
+
+import org.apache.flink.api.common.state.FoldingState;
+
+/**
+ * The peer to the {@link FoldingState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N> The type of the namespace
+ * @param <T> Type of the values folded into the state
+ * @param <ACC> Type of the value in the state
+ */
+public interface InternalFoldingState<N, T, ACC> extends InternalAppendingState<N, T, ACC>, FoldingState<T, ACC> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalKvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalKvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalKvState.java
new file mode 100644
index 0000000..06f64b6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalKvState.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.state.internal;
+
+import org.apache.flink.api.common.state.State;
+
+/**
+ * The {@code InternalKvState} is the root of the internal state type hierarchy, similar to the
+ * {@link State} being the root of the public API state hierarchy.
+ * 
+ * <p>The internal state classes give access to the namespace getters and setters and access to
+ * additional functionality, like raw value access or state merging.
+ * 
+ * <p>The public API state hierarchy is intended to be programmed against by Flink applications.
+ * The internal state hierarchy holds all the auxiliary methods that are used by the runtime and not
+ * intended to be used by user applications. These internal methods are considered of limited use to users and
+ * only confusing, and are usually not regarded as stable across releases.
+ * 
+ * <p>Each specific type in the internal state hierarchy extends the type from the public
+ * state hierarchy:
+ * 
+ * <pre>
+ *             State
+ *               |
+ *               +-------------------InternalKvState
+ *               |                         |
+ *          MergingState                   |
+ *               |                         |
+ *               +-----------------InternalMergingState
+ *               |                         |
+ *      +--------+------+                  |
+ *      |               |                  |
+ * ReducingState    ListState        +-----+-----------------+
+ *      |               |            |                       |
+ *      +-----------+   +-----------   -----------------InternalListState
+ *                  |                |
+ *                  +---------InternalReducingState
+ * </pre>
+ * 
+ * @param <N> The type of the namespace.
+ */
+public interface InternalKvState<N> extends State {
+
+	/**
+	 * Sets the current namespace, which will be used when using the state access methods.
+	 *
+	 * @param namespace The namespace.
+	 */
+	void setCurrentNamespace(N namespace);
+
+	/**
+	 * Returns the serialized value for the given key and namespace.
+	 *
+	 * <p>If no value is associated with key and namespace, <code>null</code>
+	 * is returned.
+	 *
+	 * @param serializedKeyAndNamespace Serialized key and namespace
+	 * @return Serialized value or <code>null</code> if no value is associated with the key and namespace.
+	 * 
+	 * @throws Exception Exceptions during serialization are forwarded
+	 */
+	byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java
new file mode 100644
index 0000000..ae392ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalListState.java
@@ -0,0 +1,31 @@
+/*
+ * 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.internal;
+
+import org.apache.flink.api.common.state.ListState;
+
+/**
+ * The peer to the {@link ListState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N> The type of the namespace
+ * @param <T> The type of elements in the list
+ */
+public interface InternalListState<N, T> extends InternalMergingState<N, T, Iterable<T>>, ListState<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalMergingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalMergingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalMergingState.java
new file mode 100644
index 0000000..abc7d7c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalMergingState.java
@@ -0,0 +1,46 @@
+/*
+ * 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.internal;
+
+import org.apache.flink.api.common.state.MergingState;
+
+import java.util.Collection;
+
+/**
+ * The peer to the {@link MergingState} in the internal state type hierarchy.
+ * 
+ * See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N>   The type of the namespace
+ * @param <IN>  The type of elements added to the state
+ * @param <OUT> The type of elements 
+ */
+public interface InternalMergingState<N, IN, OUT> extends InternalAppendingState<N, IN, OUT>, MergingState<IN, OUT> {
+
+	/**
+	 * Merges the state of the current key for the given source namespaces into the state of
+	 * the target namespace.
+	 * 
+	 * @param target The target namespace where the merged state should be stored.
+	 * @param sources The source namespaces whose state should be merged.
+	 * 
+	 * @throws Exception The method may forward exception thrown internally (by I/O or functions).
+	 */
+	void mergeNamespaces(N target, Collection<N> sources) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
new file mode 100644
index 0000000..40e625c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
@@ -0,0 +1,31 @@
+/*
+ * 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.internal;
+
+import org.apache.flink.api.common.state.ReducingState;
+
+/**
+ * The peer to the {@link ReducingState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N> The type of the namespace
+ * @param <T> The type of elements in the list
+ */
+public interface InternalReducingState<N, T> extends InternalMergingState<N, T, T>, ReducingState<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalValueState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalValueState.java
new file mode 100644
index 0000000..7177b8a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalValueState.java
@@ -0,0 +1,31 @@
+/*
+ * 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.internal;
+
+import org.apache.flink.api.common.state.ValueState;
+
+/**
+ * The peer to the {@link ValueState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N> The type of the namespace
+ * @param <T> The type of elements in the list
+ */
+public interface InternalValueState<N, T> extends InternalKvState<N>, ValueState<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
index 86f8766..1c02ca1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
@@ -43,8 +43,8 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
@@ -593,7 +593,7 @@ public class KvStateClientTest {
 				state.update(201 + i);
 
 				// we know it must be a KvStat but this is not exposed to the user via State
-				KvState<?> kvState = (KvState<?>) state;
+				InternalKvState<?> kvState = (InternalKvState<?>) state;
 
 				// Register KvState (one state instance for all server)
 				ids[i] = registry[i].registerKvState(new JobID(), new JobVertexID(), new KeyGroupRange(0, 0), "any", kvState);

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
index b1ec86f..202024c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
@@ -40,9 +40,9 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
@@ -267,7 +267,7 @@ public class KvStateServerHandlerTest extends TestLogger {
 	}
 
 	/**
-	 * Tests the failure response on a failure on the {@link KvState#getSerializedValue(byte[])}
+	 * Tests the failure response on a failure on the {@link InternalKvState#getSerializedValue(byte[])}
 	 * call.
 	 */
 	@Test
@@ -279,7 +279,7 @@ public class KvStateServerHandlerTest extends TestLogger {
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		// Failing KvState
-		KvState<?> kvState = mock(KvState.class);
+		InternalKvState<?> kvState = mock(InternalKvState.class);
 		when(kvState.getSerializedValue(any(byte[].class)))
 				.thenThrow(new RuntimeException("Expected test Exception"));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
index 0d9c2e4..69dbe6f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializerTest.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.query.netty.message;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.UnpooledByteBufAllocator;
-import org.apache.flink.api.common.state.ListState;
+
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
@@ -30,10 +30,12 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+
 import org.junit.Test;
 
 import java.io.IOException;
@@ -315,7 +317,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test
 	public void testListSerialization() throws Exception {
-		final long key = 0l;
+		final long key = 0L;
 
 		// objects for heap state list serialisation
 		final HeapKeyedStateBackend<Long> longHeapKeyedStateBackend =
@@ -327,9 +329,10 @@ public class KvStateRequestSerializerTest {
 			);
 		longHeapKeyedStateBackend.setCurrentKey(key);
 
-		final ListState<Long> listState = longHeapKeyedStateBackend
-			.createListState(VoidNamespaceSerializer.INSTANCE,
+		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend.createListState(
+				VoidNamespaceSerializer.INSTANCE,
 				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
+
 		testListSerialization(key, listState);
 	}
 
@@ -340,19 +343,16 @@ public class KvStateRequestSerializerTest {
 	 * @param key
 	 * 		key of the list state
 	 * @param listState
-	 * 		list state using the {@link VoidNamespace}, must also be a {@link
-	 * 		KvState} instance
+	 * 		list state using the {@link VoidNamespace}, must also be a {@link InternalKvState} instance
 	 *
 	 * @throws Exception
 	 */
-	public static void testListSerialization(final long key,
-		final ListState<Long> listState) throws Exception {
+	public static void testListSerialization(
+			final long key,
+			final InternalListState<VoidNamespace, Long> listState) throws Exception {
 
 		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
-
-		final KvState<VoidNamespace> listKvState =
-			(KvState<VoidNamespace>) listState;
-		listKvState.setCurrentNamespace(VoidNamespace.INSTANCE);
+		listState.setCurrentNamespace(VoidNamespace.INSTANCE);
 
 		// List
 		final int numElements = 10;
@@ -368,8 +368,8 @@ public class KvStateRequestSerializerTest {
 			KvStateRequestSerializer.serializeKeyAndNamespace(
 				key, LongSerializer.INSTANCE,
 				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
-		final byte[] serializedValues =
-			listKvState.getSerializedValue(serializedKey);
+		
+		final byte[] serializedValues = listState.getSerializedValue(serializedKey);
 
 		List<Long> actualValues = KvStateRequestSerializer.deserializeList(serializedValues, valueSerializer);
 		assertEquals(expectedValues, actualValues);

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/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 38e04aa..c560ab0 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
@@ -48,6 +48,7 @@ import org.apache.flink.runtime.query.KvStateRegistryListener;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.state.heap.AbstractHeapState;
 import org.apache.flink.runtime.state.heap.StateTable;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -167,7 +168,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 		@SuppressWarnings("unchecked")
-		KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+		InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
 		// some modifications to the state
 		backend.setCurrentKey(1);
@@ -214,7 +215,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		ValueState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 		@SuppressWarnings("unchecked")
-		KvState<VoidNamespace> restoredKvState1 = (KvState<VoidNamespace>) restored1;
+		InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
 		backend.setCurrentKey(1);
 		assertEquals("1", restored1.value());
@@ -230,7 +231,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 		ValueState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 		@SuppressWarnings("unchecked")
-		KvState<VoidNamespace> restoredKvState2 = (KvState<VoidNamespace>) restored2;
+		InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
 		backend.setCurrentKey(1);
 		assertEquals("u1", restored2.value());
@@ -246,7 +247,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	}
 
 	/**
-	 * Tests {@link ValueState#value()} and {@link KvState#getSerializedValue(byte[])}
+	 * Tests {@link ValueState#value()} and {@link InternalKvState#getSerializedValue(byte[])}
 	 * accessing the state concurrently. They should not get in the way of each
 	 * other.
 	 */
@@ -255,7 +256,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	public void testValueStateRace() throws Exception {
 		final AbstractKeyedStateBackend<Integer> backend =
 			createKeyedBackend(IntSerializer.INSTANCE);
-		final Integer namespace = Integer.valueOf(1);
+		final Integer namespace = 1;
 
 		final ValueStateDescriptor<String> kvId =
 			new ValueStateDescriptor<>("id", String.class);
@@ -270,7 +271,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			.getPartitionedState(namespace, IntSerializer.INSTANCE, kvId);
 
 		@SuppressWarnings("unchecked")
-		final KvState<Integer> kvState = (KvState<Integer>) state;
+		final InternalKvState<Integer> kvState = (InternalKvState<Integer>) state;
 
 		/**
 		 * 1) Test that ValueState#value() before and after
@@ -496,7 +497,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ListState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
 			Joiner joiner = Joiner.on(",");
 			// some modifications to the state
@@ -544,7 +545,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ListState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState1 = (KvState<VoidNamespace>) restored1;
+			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
 			backend.setCurrentKey(1);
 			assertEquals("1", joiner.join(restored1.get()));
@@ -560,7 +561,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ListState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState2 = (KvState<VoidNamespace>) restored2;
+			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
 			backend.setCurrentKey(1);
 			assertEquals("1,u1", joiner.join(restored2.get()));
@@ -596,7 +597,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ReducingState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
 			// some modifications to the state
 			backend.setCurrentKey(1);
@@ -643,7 +644,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ReducingState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState1 = (KvState<VoidNamespace>) restored1;
+			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
 			backend.setCurrentKey(1);
 			assertEquals("1", restored1.get());
@@ -659,7 +660,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			ReducingState<String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState2 = (KvState<VoidNamespace>) restored2;
+			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
 			backend.setCurrentKey(1);
 			assertEquals("1,u1", restored2.get());
@@ -698,7 +699,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			FoldingState<Integer, String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 
 			// some modifications to the state
 			backend.setCurrentKey(1);
@@ -746,7 +747,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 			FoldingState<Integer, String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState1 = (KvState<VoidNamespace>) restored1;
+			InternalKvState<VoidNamespace> restoredKvState1 = (InternalKvState<VoidNamespace>) restored1;
 
 			backend.setCurrentKey(1);
 			assertEquals("Fold-Initial:,1", restored1.get());
@@ -763,7 +764,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			@SuppressWarnings("unchecked")
 			FoldingState<Integer, String> restored2 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 			@SuppressWarnings("unchecked")
-			KvState<VoidNamespace> restoredKvState2 = (KvState<VoidNamespace>) restored2;
+			InternalKvState<VoidNamespace> restoredKvState2 = (InternalKvState<VoidNamespace>) restored2;
 
 			backend.setCurrentKey(1);
 			assertEquals("Fold-Initial:,101", restored2.get());
@@ -1254,7 +1255,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 					VoidNamespaceSerializer.INSTANCE,
 					desc);
 
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 			assertTrue(kvState instanceof AbstractHeapState);
 
 			kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
@@ -1280,7 +1281,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 					VoidNamespaceSerializer.INSTANCE,
 					desc);
 
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 			assertTrue(kvState instanceof AbstractHeapState);
 
 			kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
@@ -1311,7 +1312,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 					VoidNamespaceSerializer.INSTANCE,
 					desc);
 
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 			assertTrue(kvState instanceof AbstractHeapState);
 
 			kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
@@ -1342,7 +1343,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 					VoidNamespaceSerializer.INSTANCE,
 					desc);
 
-			KvState<VoidNamespace> kvState = (KvState<VoidNamespace>) state;
+			InternalKvState<VoidNamespace> kvState = (InternalKvState<VoidNamespace>) state;
 			assertTrue(kvState instanceof AbstractHeapState);
 
 			kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
@@ -1451,7 +1452,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	 * if it is not null.
 	 */
 	private static <V, K, N> V getSerializedValue(
-			KvState<N> kvState,
+			InternalKvState<N> kvState,
 			K key,
 			TypeSerializer<K> keySerializer,
 			N namespace,
@@ -1475,7 +1476,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	 * if it is not null.
 	 */
 	private static <V, K, N> List<V> getSerializedList(
-			KvState<N> kvState,
+			InternalKvState<N> kvState,
 			K key,
 			TypeSerializer<K> keySerializer,
 			N namespace,

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapListStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapListStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapListStateTest.java
new file mode 100644
index 0000000..33d60a0
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapListStateTest.java
@@ -0,0 +1,248 @@
+/*
+ * 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.heap;
+
+import org.apache.flink.api.common.ExecutionConfig;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalListState;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the simple Java heap objects implementation of the {@link ListState}.
+ */
+public class HeapListStateTest {
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final ListStateDescriptor<Long> stateDescr = new ListStateDescriptor<>("my-state", Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			InternalListState<VoidNamespace, Long> state =
+					keyedBackend.createListState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(asList(17L, 11L), state.get());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(asList(17L, 11L), state.get());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(asList(1L, 2L, 3L, 2L, 1L), state.get());
+			state.clear();
+
+			// make sure all lists / maps are cleared
+
+			StateTable<String, VoidNamespace, ArrayList<Long>> stateTable =
+					((HeapListState<String, VoidNamespace, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final ListStateDescriptor<Long> stateDescr = new ListStateDescriptor<>("my-state", Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final Integer namespace1 = 1;
+		final Integer namespace2 = 2;
+		final Integer namespace3 = 3;
+
+		final Set<Long> expectedResult = new HashSet<>(asList(11L, 22L, 33L, 44L, 55L));
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			InternalListState<Integer, Long> state = keyedBackend.createListState(IntSerializer.INSTANCE, stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(namespace2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			validateResult(state.get(), expectedResult);
+
+			// make sure all lists / maps are cleared
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("ghi");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			StateTable<String, Integer, ArrayList<Long>> stateTable = 
+					((HeapListState<String, Integer, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	private static HeapKeyedStateBackend<String> createKeyedBackend() throws Exception {
+		return new HeapKeyedStateBackend<>(
+				mock(TaskKvStateRegistry.class),
+				StringSerializer.INSTANCE,
+				HeapListStateTest.class.getClassLoader(),
+				16,
+				new KeyGroupRange(0, 15));
+	}
+	
+	private static <T> void validateResult(Iterable<T> values, Set<T> expected) {
+		int num = 0;
+		for (T v : values) {
+			num++;
+			assertTrue(expected.contains(v));
+		}
+
+		assertEquals(expected.size(), num);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapReducingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapReducingStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapReducingStateTest.java
new file mode 100644
index 0000000..e0929f1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapReducingStateTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.heap;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+
+import org.junit.Test;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the simple Java heap objects implementation of the {@link ReducingState}.
+ */
+public class HeapReducingStateTest {
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final ReducingStateDescriptor<Long> stateDescr =
+				new ReducingStateDescriptor<>("my-state", new AddingFunction(), Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			InternalReducingState<VoidNamespace, Long> state =
+					keyedBackend.createReducingState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(28L, state.get().longValue());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(28L, state.get().longValue());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(9L, state.get().longValue());
+			state.clear();
+
+			// make sure all lists / maps are cleared
+
+			StateTable<String, VoidNamespace, Long> stateTable =
+					((HeapReducingState<String, VoidNamespace, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final ReducingStateDescriptor<Long> stateDescr = new ReducingStateDescriptor<>(
+				"my-state", new AddingFunction(), Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final Integer namespace1 = 1;
+		final Integer namespace2 = 2;
+		final Integer namespace3 = 3;
+
+		final Long expectedResult = 165L;
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			final InternalReducingState<Integer, Long> state =
+					keyedBackend.createReducingState(IntSerializer.INSTANCE, stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(namespace2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			// make sure all lists / maps are cleared
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("ghi");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+			
+			StateTable<String, Integer, Long> stateTable =
+					((HeapReducingState<String, Integer, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static HeapKeyedStateBackend<String> createKeyedBackend() throws Exception {
+		return new HeapKeyedStateBackend<>(
+				mock(TaskKvStateRegistry.class),
+				StringSerializer.INSTANCE,
+				HeapReducingStateTest.class.getClassLoader(),
+				16,
+				new KeyGroupRange(0, 15));
+	}
+
+	// ------------------------------------------------------------------------
+	//  test functions
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static class AddingFunction implements ReduceFunction<Long> {
+
+		@Override
+		public Long reduce(Long a, Long b)  {
+			return a + b;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/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 a21660c..6bb0a40 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
@@ -494,15 +494,34 @@ public abstract class AbstractStreamOperator<OUT>
 		return getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, stateDescriptor);
 	}
 
+
+	protected <N, S extends State, T> S getOrCreateKeyedState(
+			TypeSerializer<N> namespaceSerializer,
+			StateDescriptor<S, T> stateDescriptor) throws Exception {
+
+		if (keyedStateStore != null) {
+			return keyedStateBackend.getOrCreateKeyedState(namespaceSerializer, stateDescriptor);
+		}
+		else {
+			throw new IllegalStateException("Cannot create partitioned state. " +
+					"The keyed state backend has not been set." +
+					"This indicates that the operator is not partitioned/keyed.");
+		}
+	}
+
 	/**
 	 * Creates a partitioned state handle, using the state backend configured for this task.
+	 * 
+	 * TODO: NOTE: This method does a lot of work caching / retrieving states just to update the namespace.
+	 *       This method should be removed for the sake of namespaces being lazily fetched from the keyed
+	 *       state backend, or being set on the state directly.
 	 *
 	 * @throws IllegalStateException Thrown, if the key/value state was already initialized.
 	 * @throws Exception Thrown, if the state backend cannot create the key/value state.
 	 */
-	@SuppressWarnings("unchecked")
 	protected <S extends State, N> S getPartitionedState(
-			N namespace, TypeSerializer<N> namespaceSerializer,
+			N namespace,
+			TypeSerializer<N> namespaceSerializer,
 			StateDescriptor<S, ?> stateDescriptor) throws Exception {
 
 		if (keyedStateStore != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
index 0d5d091..05c89dd 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
@@ -35,7 +35,7 @@ import java.util.Set;
 
 /**
  * A {@link Window} that represents a time interval from {@code start} (inclusive) to
- * {@code start + size} (exclusive).
+ * {@code end} (exclusive).
  */
 @PublicEvolving
 public class TimeWindow extends Window {
@@ -48,14 +48,35 @@ public class TimeWindow extends Window {
 		this.end = end;
 	}
 
+	/**
+	 * Gets the starting timestamp of the window. This is the first timestamp that belongs
+	 * to this window.
+	 * 
+	 * @return The starting timestamp of this window.
+	 */
 	public long getStart() {
 		return start;
 	}
 
+	/**
+	 * Gets the end timestamp of this window. The end timestamp is exclusive, meaning it
+	 * is the first timestamp that does not belong to this window any more.
+	 * 
+	 * @return The exclusive end timestamp of this window.
+	 */
 	public long getEnd() {
 		return end;
 	}
 
+	/**
+	 * Gets the largest timestamp that still belongs to this window.
+	 * 
+	 * <p>This timestamp is identical to {@code getEnd() - 1}.
+	 * 
+	 * @return The largest timestamp that still belongs to this window.
+	 * 
+	 * @see #getEnd() 
+	 */
 	@Override
 	public long maxTimestamp() {
 		return end - 1;
@@ -104,6 +125,13 @@ public class TimeWindow extends Window {
 		return new TimeWindow(Math.min(start, other.start), Math.max(end, other.end));
 	}
 
+	// ------------------------------------------------------------------------
+	// Serializer
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The serializer used to write the TimeWindow type.
+	 */
 	public static class Serializer extends TypeSerializer<TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
@@ -152,9 +180,7 @@ public class TimeWindow extends Window {
 
 		@Override
 		public TimeWindow deserialize(TimeWindow reuse, DataInputView source) throws IOException {
-			long start = source.readLong();
-			long end = source.readLong();
-			return new TimeWindow(start, end);
+			return deserialize(source);
 		}
 
 		@Override
@@ -179,6 +205,10 @@ public class TimeWindow extends Window {
 		}
 	}
 
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Merge overlapping {@link TimeWindow}s. For use by merging
 	 * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner WindowAssigners}.

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
index 0e131ff..b17989c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
@@ -31,5 +31,10 @@ import org.apache.flink.annotation.PublicEvolving;
 @PublicEvolving
 public abstract class Window {
 
+	/**
+	 * Gets the largest timestamp that still belongs to this window.
+	 *
+	 * @return The largest timestamp that still belongs to this window.
+	 */
 	public abstract long maxTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index 8c73878..d9c977a 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
@@ -20,15 +20,16 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 import com.google.common.base.Function;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.Iterables;
+
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.AppendingState;
-import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.streaming.api.operators.InternalTimer;
 import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -41,7 +42,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import java.util.Collection;
 
-import static java.util.Objects.requireNonNull;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * A {@link WindowOperator} that also allows an {@link Evictor} to be used.
@@ -56,40 +57,52 @@ import static java.util.Objects.requireNonNull;
  * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
  */
 @Internal
-public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends WindowOperator<K, IN, Iterable<IN>, OUT, W> {
+public class EvictingWindowOperator<K, IN, OUT, W extends Window> 
+		extends WindowOperator<K, IN, Iterable<IN>, OUT, W> {
 
 	private static final long serialVersionUID = 1L;
 
+	// ------------------------------------------------------------------------
+	// these fields are set by the API stream graph builder to configure the operator 
+	
 	private final Evictor<? super IN, ? super W> evictor;
 
+	private final StateDescriptor<? extends ListState<StreamRecord<IN>>, ?> evictingWindowStateDescriptor;
+
+	// ------------------------------------------------------------------------
+	// the fields below are instantiated once the operator runs in the runtime 
+
 	private transient EvictorContext evictorContext;
 
-	private final StateDescriptor<? extends ListState<StreamRecord<IN>>, ?> windowStateDescriptor;
+	private transient InternalListState<W, StreamRecord<IN>> evictingWindowState;
+
+	// ------------------------------------------------------------------------
 
 	public EvictingWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
-		TypeSerializer<W> windowSerializer,
-		KeySelector<IN, K> keySelector,
-		TypeSerializer<K> keySerializer,
-		StateDescriptor<? extends ListState<StreamRecord<IN>>, ?> windowStateDescriptor,
-		InternalWindowFunction<Iterable<IN>, OUT, K, W> windowFunction,
-		Trigger<? super IN, ? super W> trigger,
-		Evictor<? super IN, ? super W> evictor,
-		long allowedLateness) {
+			TypeSerializer<W> windowSerializer,
+			KeySelector<IN, K> keySelector,
+			TypeSerializer<K> keySerializer,
+			StateDescriptor<? extends ListState<StreamRecord<IN>>, ?> windowStateDescriptor,
+			InternalWindowFunction<Iterable<IN>, OUT, K, W> windowFunction,
+			Trigger<? super IN, ? super W> trigger,
+			Evictor<? super IN, ? super W> evictor,
+			long allowedLateness) {
 
 		super(windowAssigner, windowSerializer, keySelector,
 			keySerializer, null, windowFunction, trigger, allowedLateness);
-		this.evictor = requireNonNull(evictor);
-		this.windowStateDescriptor = windowStateDescriptor;
+
+		this.evictor = checkNotNull(evictor);
+		this.evictingWindowStateDescriptor = checkNotNull(windowStateDescriptor);
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
 	public void processElement(StreamRecord<IN> element) throws Exception {
 		Collection<W> elementWindows = windowAssigner.assignWindows(
 				element.getValue(),
 				element.getTimestamp(),
 				windowAssignerContext);
 
+		@SuppressWarnings("unchecked")
 		final K key = (K) getKeyedStateBackend().getCurrentKey();
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
@@ -119,11 +132,7 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 								}
 
 								// merge the merged state windows into the newly resulting state window
-								getKeyedStateBackend().mergePartitionedStates(
-									stateWindowResult,
-									mergedStateWindows,
-									windowSerializer,
-									(StateDescriptor<? extends MergingState<?, ?>, ?>) windowStateDescriptor);
+								evictingWindowState.mergeNamespaces(stateWindowResult, mergedStateWindows);
 							}
 						});
 
@@ -137,9 +146,9 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 				if (stateWindow == null) {
 					throw new IllegalStateException("Window " + window + " is not in in-flight window set.");
 				}
-				ListState<StreamRecord<IN>> windowState = getPartitionedState(
-					stateWindow, windowSerializer, windowStateDescriptor);
-				windowState.add(element);
+				
+				evictingWindowState.setCurrentNamespace(stateWindow);
+				evictingWindowState.add(element);
 
 				context.key = key;
 				context.window = actualWindow;
@@ -149,16 +158,16 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 				TriggerResult triggerResult = context.onElement(element);
 
 				if (triggerResult.isFire()) {
-					Iterable<StreamRecord<IN>> contents = windowState.get();
+					Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 					if (contents == null) {
 						// if we have no state, there is nothing to do
 						continue;
 					}
-					fire(actualWindow, contents, windowState);
+					fire(actualWindow, contents, evictingWindowState);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(actualWindow, windowState, mergingWindows);
+					cleanup(actualWindow, evictingWindowState, mergingWindows);
 				} else {
 					registerCleanupTimer(actualWindow);
 				}
@@ -173,9 +182,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 					continue;
 				}
 
-				ListState<StreamRecord<IN>> windowState = getPartitionedState(
-					window, windowSerializer, windowStateDescriptor);
-				windowState.add(element);
+				evictingWindowState.setCurrentNamespace(window);
+				evictingWindowState.add(element);
 
 				context.key = key;
 				context.window = window;
@@ -185,16 +193,16 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 				TriggerResult triggerResult = context.onElement(element);
 
 				if (triggerResult.isFire()) {
-					Iterable<StreamRecord<IN>> contents = windowState.get();
+					Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 					if (contents == null) {
 						// if we have no state, there is nothing to do
 						continue;
 					}
-					fire(window, contents, windowState);
+					fire(window, contents, evictingWindowState);
 				}
 
 				if (triggerResult.isPurge()) {
-					cleanup(window, windowState, null);
+					cleanup(window, evictingWindowState, null);
 				} else {
 					registerCleanupTimer(window);
 				}
@@ -222,15 +230,13 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 				// so it is safe to just ignore
 				return;
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			
+			evictingWindowState.setCurrentNamespace(stateWindow);
 		} else {
-			windowState = getPartitionedState(
-					context.window,
-					windowSerializer,
-					windowStateDescriptor);
+			evictingWindowState.setCurrentNamespace(context.window);
 		}
 
-		Iterable<StreamRecord<IN>> contents = windowState.get();
+		Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 		if (contents == null) {
 			// if we have no state, there is nothing to do
 			return;
@@ -238,11 +244,11 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 
 		TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
 		if (triggerResult.isFire()) {
-			fire(context.window, contents, windowState);
+			fire(context.window, contents, evictingWindowState);
 		}
 
 		if (triggerResult.isPurge() || (windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+			cleanup(context.window, evictingWindowState, mergingWindows);
 		}
 	}
 
@@ -265,12 +271,12 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 				// so it is safe to just ignore
 				return;
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			evictingWindowState.setCurrentNamespace(stateWindow);
 		} else {
-			windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
+			evictingWindowState.setCurrentNamespace(context.window);
 		}
 
-		Iterable<StreamRecord<IN>> contents = windowState.get();
+		Iterable<StreamRecord<IN>> contents = evictingWindowState.get();
 		if (contents == null) {
 			// if we have no state, there is nothing to do
 			return;
@@ -278,11 +284,11 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 
 		TriggerResult triggerResult = context.onProcessingTime(timer.getTimestamp());
 		if (triggerResult.isFire()) {
-			fire(context.window, contents, windowState);
+			fire(context.window, contents, evictingWindowState);
 		}
 
 		if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.getTimestamp()))) {
-			cleanup(context.window, windowState, mergingWindows);
+			cleanup(context.window, evictingWindowState, mergingWindows);
 		}
 	}
 
@@ -381,7 +387,10 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 	@Override
 	public void open() throws Exception {
 		super.open();
+
 		evictorContext = new EvictorContext(null,null);
+		evictingWindowState = (InternalListState<W, StreamRecord<IN>>) 
+				getOrCreateKeyedState(windowSerializer, evictingWindowStateDescriptor);
 	}
 
 	@Override
@@ -409,6 +418,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 	@VisibleForTesting
 	@SuppressWarnings("unchecked, rawtypes")
 	public StateDescriptor<? extends AppendingState<IN, Iterable<IN>>, ?> getStateDescriptor() {
-		return (StateDescriptor<? extends AppendingState<IN, Iterable<IN>>, ?>) windowStateDescriptor;
+		return (StateDescriptor<? extends AppendingState<IN, Iterable<IN>>, ?>) evictingWindowStateDescriptor;
 	}
 }


[2/9] flink git commit: [hotfix] Remove no longer used Generic State classes

Posted by se...@apache.org.
[hotfix] Remove no longer used Generic State classes


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

Branch: refs/heads/master
Commit: 254a70072aa3bbd51277e99e0982c5e29908d684
Parents: 51c02ee
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jan 13 14:52:40 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:22:21 2017 +0100

----------------------------------------------------------------------
 .../runtime/state/GenericFoldingState.java      | 80 -------------------
 .../flink/runtime/state/GenericListState.java   | 84 --------------------
 .../runtime/state/GenericReducingState.java     | 83 -------------------
 3 files changed, 247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/254a7007/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericFoldingState.java
deleted file mode 100644
index ee2d86d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericFoldingState.java
+++ /dev/null
@@ -1,80 +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.state;
-
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.ValueState;
-
-/**
- * Generic implementation of {@link FoldingState} based on a wrapped {@link ValueState}.
- *
- * @param <N> The type of the namespace.
- * @param <T> The type of the values that can be folded into the state.
- * @param <ACC> The type of the value in the folding state.
- * @param <W> Generic type that extends both the underlying {@code ValueState} and {@code KvState}.
- */
-public class GenericFoldingState<N, T, ACC, W extends ValueState<ACC> & KvState<N>>
-	implements FoldingState<T, ACC>, KvState<N> {
-
-	private final W wrappedState;
-	private final FoldFunction<T, ACC> foldFunction;
-
-	/**
-	 * Creates a new {@code FoldingState} that wraps the given {@link ValueState}. The
-	 * {@code ValueState} must have the initial value of the fold as default value.
-	 *
-	 * @param wrappedState The wrapped {@code ValueState}
-	 * @param foldFunction The {@code FoldFunction} to use for folding values into the state
-	 */
-	@SuppressWarnings("unchecked")
-	public GenericFoldingState(ValueState<ACC> wrappedState, FoldFunction<T, ACC> foldFunction) {
-		if (!(wrappedState instanceof KvState)) {
-			throw new IllegalArgumentException("Wrapped state must be a KvState.");
-		}
-		this.wrappedState = (W) wrappedState;
-		this.foldFunction = foldFunction;
-	}
-
-	@Override
-	public void setCurrentNamespace(N namespace) {
-		wrappedState.setCurrentNamespace(namespace);
-	}
-
-	@Override
-	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
-		return wrappedState.getSerializedValue(serializedKeyAndNamespace);
-	}
-
-	@Override
-	public ACC get() throws Exception {
-		return wrappedState.value();
-	}
-
-	@Override
-	public void add(T value) throws Exception {
-		ACC currentValue = wrappedState.value();
-		wrappedState.update(foldFunction.fold(currentValue, value));
-	}
-
-	@Override
-	public void clear() {
-		wrappedState.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/254a7007/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java
deleted file mode 100644
index ba81837..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java
+++ /dev/null
@@ -1,84 +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.state;
-
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ValueState;
-
-import java.util.ArrayList;
-
-/**
- * Generic implementation of {@link ListState} based on a wrapped {@link ValueState}.
- *
- * @param <N> The type of the namespace.
- * @param <T> The type of the values stored in this {@code ListState}.
- * @param <W> Generic type that extends both the underlying {@code ValueState} and {@code KvState}.
- */
-public class GenericListState<N, T, W extends ValueState<ArrayList<T>> & KvState<N>>
-	implements ListState<T>, KvState<N> {
-
-	private final W wrappedState;
-
-	/**
-	 * Creates a new {@code ListState} that wraps the given {@link ValueState}. The
-	 * {@code ValueState} must have a default value of {@code null}.
-	 *
-	 * @param wrappedState The wrapped {@code ValueState}
-	 */
-	@SuppressWarnings("unchecked")
-	public GenericListState(ValueState<ArrayList<T>> wrappedState) {
-		if (!(wrappedState instanceof KvState)) {
-			throw new IllegalArgumentException("Wrapped state must be a KvState.");
-		}
-		this.wrappedState = (W) wrappedState;
-	}
-
-	@Override
-	public void setCurrentNamespace(N namespace) {
-		wrappedState.setCurrentNamespace(namespace);
-	}
-
-	@Override
-	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
-		return wrappedState.getSerializedValue(serializedKeyAndNamespace);
-	}
-
-	@Override
-	public Iterable<T> get() throws Exception {
-		return wrappedState.value();
-	}
-
-	@Override
-	public void add(T value) throws Exception {
-		ArrayList<T> currentValue = wrappedState.value();
-		if (currentValue == null) {
-			currentValue = new ArrayList<>();
-			currentValue.add(value);
-			wrappedState.update(currentValue);
-		} else {
-			currentValue.add(value);
-			wrappedState.update(currentValue);
-		}
-	}
-
-	@Override
-	public void clear() {
-		wrappedState.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/254a7007/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericReducingState.java
deleted file mode 100644
index 214231e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericReducingState.java
+++ /dev/null
@@ -1,83 +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.state;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ValueState;
-
-/**
- * Generic implementation of {@link ReducingState} based on a wrapped {@link ValueState}.
- *
- * @param <N> The type of the namespace.
- * @param <T> The type of the values stored in this {@code ReducingState}.
- * @param <W> Generic type that extends both the underlying {@code ValueState} and {@code KvState}.
- */
-public class GenericReducingState<N, T, W extends ValueState<T> & KvState<N>>
-	implements ReducingState<T>, KvState<N> {
-
-	private final W wrappedState;
-	private final ReduceFunction<T> reduceFunction;
-
-	/**
-	 * Creates a new {@code ReducingState} that wraps the given {@link ValueState}. The
-	 * {@code ValueState} must have a default value of {@code null}.
-	 *
-	 * @param wrappedState The wrapped {@code ValueState}
-	 * @param reduceFunction The {@code ReduceFunction} to use for combining values.
-	 */
-	@SuppressWarnings("unchecked")
-	public GenericReducingState(ValueState<T> wrappedState, ReduceFunction<T> reduceFunction) {
-		if (!(wrappedState instanceof KvState)) {
-			throw new IllegalArgumentException("Wrapped state must be a KvState.");
-		}
-		this.wrappedState = (W) wrappedState;
-		this.reduceFunction = reduceFunction;
-	}
-
-	@Override
-	public void setCurrentNamespace(N namespace) {
-		wrappedState.setCurrentNamespace(namespace);
-	}
-
-	@Override
-	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
-		return wrappedState.getSerializedValue(serializedKeyAndNamespace);
-	}
-
-	@Override
-	public T get() throws Exception {
-		return wrappedState.value();
-	}
-
-	@Override
-	public void add(T value) throws Exception {
-		T currentValue = wrappedState.value();
-		if (currentValue == null) {
-			wrappedState.update(value);
-		} else {
-			wrappedState.update(reduceFunction.reduce(currentValue, value));
-		}
-	}
-
-	@Override
-	public void clear() {
-		wrappedState.clear();
-	}
-}


[4/9] flink git commit: [hotfix] [runtime] Various code cleanups and reductions of warnings in heap state restoring code

Posted by se...@apache.org.
[hotfix] [runtime] Various code cleanups and reductions of warnings in heap state restoring code


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

Branch: refs/heads/master
Commit: b8a784e93811a71f525070cee8ff32230fee8fee
Parents: 3b97128
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jan 20 14:41:35 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:53:39 2017 +0100

----------------------------------------------------------------------
 .../state/heap/HeapKeyedStateBackend.java       |  7 ++-
 .../flink/runtime/state/heap/StateTable.java    | 60 +++++++++++++-------
 2 files changed, 44 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b8a784e9/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 89d4f76..b05b874 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
@@ -239,6 +239,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 	}
 
+	@SuppressWarnings("deprecation")
 	@Override
 	public void restore(Collection<KeyGroupsStateHandle> restoredState) throws Exception {
 		LOG.info("Initializing heap keyed state backend from snapshot.");
@@ -388,6 +389,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		return "HeapKeyedStateBackend";
 	}
 
+	@SuppressWarnings({"unchecked", "rawtypes", "deprecation"})
 	@Deprecated
 	private void restoreOldSavepointKeyedState(
 			Collection<KeyGroupsStateHandle> stateHandles) throws IOException, ClassNotFoundException {
@@ -447,13 +449,14 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 							stateSerializer);
 
 			StateTable<K, ?, ?> stateTable = new StateTable<>(registeredBackendStateMetaInfo, keyGroupRange);
-			stateTable.getState().set(0, rawResultMap);
+			stateTable.getState()[0] = rawResultMap;
 
 			// add named state to the backend
 			stateTables.put(registeredBackendStateMetaInfo.getName(), stateTable);
 		}
 	}
 
+	@SuppressWarnings("deprecation")
 	private RestoredState restoreHeapState(AbstractMemStateSnapshot<K, ?, ?, ?, ?> stateSnapshot) throws IOException {
 		return new RestoredState(
 				stateSnapshot.deserialize(),
@@ -461,6 +464,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				stateSnapshot.getStateSerializer());
 	}
 
+	@SuppressWarnings({"rawtypes", "unchecked", "deprecation"})
 	private RestoredState restoreFsState(AbstractFsStateSnapshot<K, ?, ?, ?, ?> stateSnapshot) throws IOException {
 		FileSystem fs = stateSnapshot.getFilePath().getFileSystem();
 		//TODO register closeable to support fast cancelation?
@@ -492,6 +496,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 	}
 
+	@SuppressWarnings("rawtypes")
 	static final class RestoredState {
 
 		private final Map rawResultMap;

http://git-wip-us.apache.org/repos/asf/flink/blob/b8a784e9/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
index 9d7232e..21265f4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java
@@ -22,44 +22,64 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.KeyGroupRange;
 
-import java.util.Arrays;
-import java.util.List;
 import java.util.Map;
 
 public class StateTable<K, N, ST> {
 
-	/** Combined meta information such as name and serializers for this state */
-	protected RegisteredBackendStateMetaInfo<N, ST> metaInfo;
-
 	/** Map for holding the actual state objects. */
-	private final List<Map<N, Map<K, ST>>> state;
+	private final Map<N, Map<K, ST>>[] state;
 
-	protected final KeyGroupRange keyGroupRange;
+	/** The offset to the contiguous key groups */
+	private final int keyGroupOffset;
 
-	public StateTable(
-			RegisteredBackendStateMetaInfo<N, ST> metaInfo,
-			KeyGroupRange keyGroupRange) {
+	/** Combined meta information such as name and serializers for this state */
+	private RegisteredBackendStateMetaInfo<N, ST> metaInfo;
+
+	// ------------------------------------------------------------------------
+	public StateTable(RegisteredBackendStateMetaInfo<N, ST> metaInfo, KeyGroupRange keyGroupRange) {
 		this.metaInfo = metaInfo;
-		this.keyGroupRange = keyGroupRange;
+		this.keyGroupOffset = keyGroupRange.getStartKeyGroup();
 
-		this.state = Arrays.asList((Map<N, Map<K, ST>>[]) new Map[keyGroupRange.getNumberOfKeyGroups()]);
+		@SuppressWarnings("unchecked")
+		Map<N, Map<K, ST>>[] state = (Map<N, Map<K, ST>>[]) new Map[keyGroupRange.getNumberOfKeyGroups()];
+		this.state = state;
 	}
 
-	private int indexToOffset(int index) {
-		return index - keyGroupRange.getStartKeyGroup();
+	// ------------------------------------------------------------------------
+	//  access to maps
+	// ------------------------------------------------------------------------
+
+	public Map<N, Map<K, ST>>[] getState() {
+		return state;
 	}
 
 	public Map<N, Map<K, ST>> get(int index) {
-		return keyGroupRange.contains(index) ? state.get(indexToOffset(index)) : null;
+		final int pos = indexToOffset(index);
+		if (pos >= 0 && pos < state.length) {
+			return state[pos];
+		} else {
+			return null;
+		}
 	}
 
 	public void set(int index, Map<N, Map<K, ST>> map) {
-		if (!keyGroupRange.contains(index)) {
-			throw new RuntimeException("Unexpected key group index. This indicates a bug.");
+		try {
+			state[indexToOffset(index)] = map;
+		}
+		catch (ArrayIndexOutOfBoundsException e) {
+			throw new IllegalArgumentException("Key group index out of range of key group range [" +
+					keyGroupOffset + ", " + (keyGroupOffset + state.length) + ").");
 		}
-		state.set(indexToOffset(index), map);
 	}
 
+	private int indexToOffset(int index) {
+		return index - keyGroupOffset;
+	}
+
+	// ------------------------------------------------------------------------
+	//  metadata
+	// ------------------------------------------------------------------------
+	
 	public TypeSerializer<ST> getStateSerializer() {
 		return metaInfo.getStateSerializer();
 	}
@@ -76,10 +96,6 @@ public class StateTable<K, N, ST> {
 		this.metaInfo = metaInfo;
 	}
 
-	public List<Map<N, Map<K, ST>>> getState() {
-		return state;
-	}
-
 	// ------------------------------------------------------------------------
 	//  for testing
 	// ------------------------------------------------------------------------


[9/9] flink git commit: [FLINK-5582] [streaming] Add 'AggregateFunction' and 'AggregatingState'.

Posted by se...@apache.org.
[FLINK-5582] [streaming] Add 'AggregateFunction' and 'AggregatingState'.

The AggregateFunction implements a very flexible interface for distributive aggregations.


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

Branch: refs/heads/master
Commit: 09380e49256bff924734b9a932808e0f4daa7e5c
Parents: 2b3fd39
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Jan 10 19:24:49 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:53:40 2017 +0100

----------------------------------------------------------------------
 .../state/RocksDBAggregatingState.java          | 205 ++++++++++++++
 .../state/RocksDBKeyedStateBackend.java         |  11 +
 .../state/RocksDBAggregatingStateTest.java      | 248 +++++++++++++++++
 .../state/RocksDBStateBackendConfigTest.java    |   4 +-
 .../api/common/functions/AggregateFunction.java |  94 +++++++
 .../api/common/state/AggregatingState.java      |  45 +++
 .../state/AggregatingStateDescriptor.java       | 145 ++++++++++
 .../flink/api/common/state/StateBackend.java    |  13 +-
 .../flink/api/common/state/StateDescriptor.java |  10 +-
 .../flink/api/java/typeutils/TypeExtractor.java |  23 ++
 .../apache/flink/core/fs/CloseableRegistry.java |   2 +
 .../memory/ByteArrayInputStreamWithPos.java     |   2 +
 .../state/AbstractKeyedStateBackend.java        |  23 ++
 .../state/heap/HeapAggregatingState.java        | 147 ++++++++++
 .../state/heap/HeapKeyedStateBackend.java       |  13 +
 .../runtime/state/heap/HeapReducingState.java   |   2 +-
 .../internal/InternalAggregatingState.java      |  33 +++
 .../state/internal/InternalReducingState.java   |   2 +-
 .../runtime/state/SerializationProxiesTest.java |   3 +-
 .../state/heap/HeapAggregatingStateTest.java    | 274 +++++++++++++++++++
 .../api/datastream/WindowedStream.java          | 177 +++++++++++-
 .../AggregateApplyAllWindowFunction.java        |  56 ++++
 .../windowing/AggregateApplyWindowFunction.java |  52 ++++
 23 files changed, 1575 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java
new file mode 100644
index 0000000..1f306b4
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.contrib.streaming.state;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteOptions;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * An {@link AggregatingState} implementation that stores state in RocksDB.
+ *
+ * @param <K> The type of the key
+ * @param <N> The type of the namespace
+ * @param <T> The type of the values that aggregated into the state
+ * @param <ACC> The type of the value stored in the state (the accumulator type)
+ * @param <R> The type of the value returned from the state
+ */
+public class RocksDBAggregatingState<K, N, T, ACC, R>
+	extends AbstractRocksDBState<K, N, AggregatingState<T, R>, AggregatingStateDescriptor<T, ACC, R>, ACC>
+	implements InternalAggregatingState<N, T, R> {
+
+	/** Serializer for the values */
+	private final TypeSerializer<ACC> valueSerializer;
+
+	/** User-specified aggregation function */
+	private final AggregateFunction<T, ACC, R> aggFunction;
+
+	/**
+	 * We disable writes to the write-ahead-log here. We can't have these in the base class
+	 * because JNI segfaults for some reason if they are.
+	 */
+	private final WriteOptions writeOptions;
+
+	/**
+	 * Creates a new {@code RocksDBFoldingState}.
+	 *
+	 * @param namespaceSerializer
+	 *             The serializer for the namespace.
+	 * @param stateDesc              
+	 *             The state identifier for the state. This contains the state name and aggregation function.
+	 */
+	public RocksDBAggregatingState(
+			ColumnFamilyHandle columnFamily,
+			TypeSerializer<N> namespaceSerializer,
+			AggregatingStateDescriptor<T, ACC, R> stateDesc,
+			RocksDBKeyedStateBackend<K> backend) {
+
+		super(columnFamily, namespaceSerializer, stateDesc, backend);
+
+		this.valueSerializer = stateDesc.getSerializer();
+		this.aggFunction = stateDesc.getAggregateFunction();
+
+		writeOptions = new WriteOptions();
+		writeOptions.setDisableWAL(true);
+	}
+
+	@Override
+	public R get() throws IOException {
+		try {
+			// prepare the current key and namespace for RocksDB lookup
+			writeCurrentKeyWithGroupAndNamespace();
+			final byte[] key = keySerializationStream.toByteArray();
+
+			// get the current value
+			final byte[] valueBytes = backend.db.get(columnFamily, key);
+
+			if (valueBytes == null) {
+				return null;
+			}
+
+			ACC accumulator = valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes)));
+			return aggFunction.getResult(accumulator);
+		}
+		catch (IOException | RocksDBException e) {
+			throw new IOException("Error while retrieving value from RocksDB", e);
+		}
+	}
+
+	@Override
+	public void add(T value) throws IOException {
+		try {
+			// prepare the current key and namespace for RocksDB lookup
+			writeCurrentKeyWithGroupAndNamespace();
+			final byte[] key = keySerializationStream.toByteArray();
+			keySerializationStream.reset();
+
+			// get the current value
+			final byte[] valueBytes = backend.db.get(columnFamily, key);
+
+			// deserialize the current accumulator, or create a blank one
+			final ACC accumulator = valueBytes == null ?
+					aggFunction.createAccumulator() :
+					valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes)));
+
+			// aggregate the value into the accumulator
+			aggFunction.add(value, accumulator);
+
+			// serialize the new accumulator
+			final DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(keySerializationStream);
+			valueSerializer.serialize(accumulator, out);
+
+			// write the new value to RocksDB
+			backend.db.put(columnFamily, writeOptions, key, keySerializationStream.toByteArray());
+		}
+		catch (IOException | RocksDBException e) {
+			throw new IOException("Error while adding value to RocksDB", e);
+		}
+	}
+
+	@Override
+	public void mergeNamespaces(N target, Collection<N> sources) throws Exception {
+		if (sources == null || sources.isEmpty()) {
+			return;
+		}
+
+		// cache key and namespace
+		final K key = backend.getCurrentKey();
+		final int keyGroup = backend.getCurrentKeyGroupIndex();
+
+		try {
+			ACC current = null;
+
+			// merge the sources to the target
+			for (N source : sources) {
+				if (source != null) {
+					writeKeyWithGroupAndNamespace(
+							keyGroup, key, source,
+							keySerializationStream, keySerializationDataOutputView);
+					
+					final byte[] sourceKey = keySerializationStream.toByteArray();
+					final byte[] valueBytes = backend.db.get(columnFamily, sourceKey);
+
+					if (valueBytes != null) {
+						ACC value = valueSerializer.deserialize(
+								new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes)));
+
+						if (current != null) {
+							current = aggFunction.merge(current, value);
+						}
+						else {
+							current = value;
+						}
+					}
+				}
+			}
+
+			// if something came out of merging the sources, merge it or write it to the target
+			if (current != null) {
+				// create the target full-binary-key 
+				writeKeyWithGroupAndNamespace(
+						keyGroup, key, target,
+						keySerializationStream, keySerializationDataOutputView);
+
+				final byte[] targetKey = keySerializationStream.toByteArray();
+				final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey);
+
+				if (targetValueBytes != null) {
+					// target also had a value, merge
+					ACC value = valueSerializer.deserialize(
+							new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(targetValueBytes)));
+
+					current = aggFunction.merge(current, value);
+				}
+
+				// serialize the resulting value
+				keySerializationStream.reset();
+				valueSerializer.serialize(current, keySerializationDataOutputView);
+
+				// write the resulting value
+				backend.db.put(columnFamily, writeOptions, targetKey, keySerializationStream.toByteArray());
+			}
+		}
+		catch (Exception e) {
+			throw new Exception("Error while merging state in RocksDB", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/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 5a6d102..21ef8c2 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
@@ -18,6 +18,7 @@
 package org.apache.flink.contrib.streaming.state;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
@@ -48,6 +49,7 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
 import org.apache.flink.runtime.state.internal.InternalFoldingState;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalReducingState;
@@ -842,6 +844,15 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
+	protected <N, T, ACC, R> InternalAggregatingState<N, T, R> createAggregatingState(
+			TypeSerializer<N> namespaceSerializer,
+			AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
+
+		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
+		return new RocksDBAggregatingState<>(columnFamily, namespaceSerializer, stateDesc, this);
+	}
+
+	@Override
 	protected <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
 			TypeSerializer<N> namespaceSerializer,
 			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingStateTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingStateTest.java
new file mode 100644
index 0000000..983e569
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingStateTest.java
@@ -0,0 +1,248 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static java.util.Arrays.asList;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the {@link InternalAggregatingState} implementation on top of RocksDB.
+ */
+public class RocksDBAggregatingStateTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final AggregatingStateDescriptor<Long, MutableLong, Long> stateDescr =
+				new AggregatingStateDescriptor<>("my-state", new AddingFunction(), MutableLong.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+
+		try {
+			InternalAggregatingState<VoidNamespace, Long, Long> state =
+					keyedBackend.createAggregatingState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(28L, state.get().longValue());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(28L, state.get().longValue());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(9L, state.get().longValue());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final AggregatingStateDescriptor<Long, MutableLong, Long> stateDescr =
+				new AggregatingStateDescriptor<>("my-state", new AddingFunction(), MutableLong.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final TimeWindow win1 = new TimeWindow(1000, 2000);
+		final TimeWindow win2 = new TimeWindow(2000, 3000);
+		final TimeWindow win3 = new TimeWindow(3000, 4000);
+
+		final Long expectedResult = 165L;
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+
+		try {
+			InternalAggregatingState<TimeWindow, Long, Long> state =
+					keyedBackend.createAggregatingState(new TimeWindow.Serializer(), stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(win1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(win2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(win3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(win3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(win3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	private static RocksDBKeyedStateBackend<String> createKeyedBackend(RocksDBStateBackend backend) throws Exception {
+		return (RocksDBKeyedStateBackend<String>) backend.createKeyedStateBackend(
+						new DummyEnvironment("TestTask", 1, 0),
+						new JobID(),
+						"test-op",
+						StringSerializer.INSTANCE,
+						16,
+						new KeyGroupRange(2, 3),
+						mock(TaskKvStateRegistry.class));
+	}
+
+	//  test functions
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static class AddingFunction implements AggregateFunction<Long, MutableLong, Long> {
+
+		@Override
+		public MutableLong createAccumulator() {
+			return new MutableLong();
+		}
+
+		@Override
+		public void add(Long value, MutableLong accumulator) {
+			accumulator.value += value;
+		}
+
+		@Override
+		public Long getResult(MutableLong accumulator) {
+			return accumulator.value;
+		}
+
+		@Override
+		public MutableLong merge(MutableLong a, MutableLong b) {
+			a.value += b.value;
+			return a;
+		}
+	}
+
+	private static final class MutableLong {
+		long value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
index bc0777c..9524352 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
@@ -351,11 +351,11 @@ public class RocksDBStateBackendConfigTest {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	private static Environment getMockEnvironment() {
+	static Environment getMockEnvironment() {
 		return getMockEnvironment(new File[] { new File(System.getProperty("java.io.tmpdir")) });
 	}
 
-	private static Environment getMockEnvironment(File[] tempDirs) {
+	static Environment getMockEnvironment(File[] tempDirs) {
 		final String[] tempDirStrings = new String[tempDirs.length];
 		for (int i = 0; i < tempDirs.length; i++) {
 			tempDirStrings[i] = tempDirs[i].getAbsolutePath();

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/common/functions/AggregateFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/AggregateFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/AggregateFunction.java
new file mode 100644
index 0000000..507be63
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/AggregateFunction.java
@@ -0,0 +1,94 @@
+/*
+ * 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.api.common.functions;
+
+import java.io.Serializable;
+
+/**
+ * 
+ * <p>Aggregation functions must be {@link Serializable} because they are sent around
+ * between distributed processes during distributed execution.
+ * 
+ * <p>An example how to use this interface is below:
+ * 
+ * <pre>{@code
+ * // the accumulator, which holds the state of the in-flight aggregate
+ * public class AverageAccumulator {
+ *     long count;
+ *     long sum;
+ * }
+ * 
+ * // implementation of an aggregation function for an 'average'
+ * public class Average implements AggregateFunction<Integer, AverageAccumulator, Double> {
+ * 
+ *     public AverageAccumulator createAccumulator() {
+ *         return new AverageAccumulator();
+ *     }
+ * 
+ *     public AverageAccumulator merge(AverageAccumulator a, AverageAccumulator b) {
+ *         a.count += b.count;
+ *         a.sum += b.sum;
+ *         return a;
+ *     }
+ * 
+ *     public void add(Integer value, AverageAccumulator acc) {
+ *         acc.sum += value;
+ *         acc.count++;
+ *     }
+ * 
+ *     public Double getResult(AverageAccumulator acc) {
+ *         return acc.sum / (double) acc.count;
+ *     }
+ * }
+ * 
+ * // implementation of a weighted average
+ * // this reuses the same accumulator type as the aggregate function for 'average'
+ * public class WeightedAverage implements AggregateFunction<Datum, AverageAccumulator, Double> {
+ *
+ *     public AverageAccumulator createAccumulator() {
+ *         return new AverageAccumulator();
+ *     }
+ *
+ *     public AverageAccumulator merge(AverageAccumulator a, AverageAccumulator b) {
+ *         a.count += b.count;
+ *         a.sum += b.sum;
+ *         return a;
+ *     }
+ *
+ *     public void add(Datum value, AverageAccumulator acc) {
+ *         acc.count += value.getWeight();
+ *         acc.sum += value.getValue();
+ *     }
+ *
+ *     public Double getResult(AverageAccumulator acc) {
+ *         return acc.sum / (double) acc.count;
+ *     }
+ * }
+ * }</pre>
+ */
+public interface AggregateFunction<IN, ACC, OUT> extends Function, Serializable {
+
+	ACC createAccumulator();
+
+	void add(IN value, ACC accumulator);
+
+	OUT getResult(ACC accumulator);
+
+	ACC merge(ACC a, ACC b);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java
new file mode 100644
index 0000000..c679285
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.AggregateFunction;
+
+/**
+ * {@link State} interface for aggregating state, based on an 
+ * {@link AggregateFunction}. Elements that are added to this type of state will
+ * be eagerly pre-aggregated using a given {@code AggregateFunction}.
+ * 
+ * <p>The state holds internally always the accumulator type of the {@code AggregateFunction}.
+ * When accessing the result of the state, the function's 
+ * {@link AggregateFunction#getResult(Object)} method.
+ *
+ * <p>The state is accessed and modified by user functions, and checkpointed consistently
+ * by the system as part of the distributed snapshots.
+ * 
+ * <p>The state is only accessible by functions applied on a KeyedDataStream. The key is
+ * automatically supplied by the system, so the function always sees the value mapped to the
+ * key of the current element. That way, the system can handle stream and state partitioning
+ * consistently together.
+ * 
+ * @param <IN> Type of the value added to the state.
+ * @param <OUT> Type of the value extracted from the state.
+ */
+@PublicEvolving
+public interface AggregatingState<IN, OUT> extends MergingState<IN, OUT> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java
new file mode 100644
index 0000000..abdac91
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java
@@ -0,0 +1,145 @@
+/*
+ * 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.api.common.state;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A StateDescriptor for {@link AggregatingState}.
+ *
+ * <p>The type internally stored in the state is the type of the {@code Accumulator} of the
+ * {@code AggregateFunction}.
+ * 
+ * @param <IN> The type of the values that are added to the state.
+ * @param <ACC> The type of the accumulator (intermediate aggregation state).
+ * @param <OUT> The type of the values that are returned from the state.
+ */
+@PublicEvolving
+public class AggregatingStateDescriptor<IN, ACC, OUT> extends StateDescriptor<AggregatingState<IN, OUT>, ACC> {
+	private static final long serialVersionUID = 1L;
+
+	/** The aggregation function for the state */
+	private final AggregateFunction<IN, ACC, OUT> aggFunction;
+
+	/**
+	 * Creates a new state descriptor with the given name, function, and type.
+	 *
+	 * <p>If this constructor fails (because it is not possible to describe the type via a class),
+	 * consider using the {@link #AggregatingStateDescriptor(String, AggregateFunction, TypeInformation)} constructor.
+	 *
+	 * @param name The (unique) name for the state.
+	 * @param aggFunction The {@code AggregateFunction} used to aggregate the state.   
+	 * @param stateType The type of the accumulator. The accumulator is stored in the state.
+	 */
+	public AggregatingStateDescriptor(
+			String name,
+			AggregateFunction<IN, ACC, OUT> aggFunction,
+			Class<ACC> stateType) {
+
+		super(name, stateType, null);
+		this.aggFunction = checkNotNull(aggFunction);
+	}
+
+	/**
+	 * Creates a new {@code ReducingStateDescriptor} with the given name and default value.
+	 *
+	 * @param name The (unique) name for the state.
+	 * @param aggFunction The {@code AggregateFunction} used to aggregate the state.
+	 * @param stateType The type of the accumulator. The accumulator is stored in the state.
+	 */
+	public AggregatingStateDescriptor(
+			String name,
+			AggregateFunction<IN, ACC, OUT> aggFunction,
+			TypeInformation<ACC> stateType) {
+
+		super(name, stateType, null);
+		this.aggFunction = checkNotNull(aggFunction);
+	}
+
+	/**
+	 * Creates a new {@code ValueStateDescriptor} with the given name and default value.
+	 *
+	 * @param name The (unique) name for the state.
+	 * @param aggFunction The {@code AggregateFunction} used to aggregate the state.
+	 * @param typeSerializer The serializer for the accumulator. The accumulator is stored in the state.
+	 */
+	public AggregatingStateDescriptor(
+			String name,
+			AggregateFunction<IN, ACC, OUT> aggFunction,
+			TypeSerializer<ACC> typeSerializer) {
+
+		super(name, typeSerializer, null);
+		this.aggFunction = checkNotNull(aggFunction);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public AggregatingState<IN, OUT> bind(StateBackend stateBackend) throws Exception {
+		return stateBackend.createAggregatingState(this);
+	}
+
+	/**
+	 * Returns the aggregate function to be used for the state.
+	 */
+	public AggregateFunction<IN, ACC, OUT> getAggregateFunction() {
+		return aggFunction;
+	}
+
+	@Override
+	public Type getType() {
+		return Type.AGGREGATING;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		else if (o != null && getClass() == o.getClass()) {
+			AggregatingStateDescriptor<?, ?, ?> that = (AggregatingStateDescriptor<?, ?, ?>) o;
+			return serializer.equals(that.serializer) && name.equals(that.name);
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		int result = serializer.hashCode();
+		result = 31 * result + name.hashCode();
+		return result;
+	}
+
+	@Override
+	public String toString() {
+		return "AggregatingStateDescriptor{" +
+				"serializer=" + serializer +
+				", aggFunction=" + aggFunction +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/common/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBackend.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBackend.java
index d396a31..f9d1af7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBackend.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBackend.java
@@ -47,11 +47,22 @@ public interface StateBackend {
 	 * Creates and returns a new {@link ReducingState}.
 	 * @param stateDesc The {@code StateDescriptor} that contains the name of the state.
 	 *
-	 * @param <T> The type of the values that the {@code ListState} can store.
+	 * @param <T> The type of the values that the {@code ReducingState} can store.
 	 */
 	<T> ReducingState<T> createReducingState(ReducingStateDescriptor<T> stateDesc) throws Exception;
 
 	/**
+	 * Creates and returns a new {@link AggregatingState}.
+	 * @param stateDesc The {@code StateDescriptor} that contains the name of the state.
+	 *
+	 * @param <IN> The type of the values that go into the aggregating state
+	 * @param <ACC> The type of the values that are stored in the aggregating state   
+	 * @param <OUT> The type of the values that come out of the aggregating state   
+	 */
+	<IN, ACC, OUT> AggregatingState<IN, OUT> createAggregatingState(
+			AggregatingStateDescriptor<IN, ACC, OUT> stateDesc) throws Exception;
+
+	/**
 	 * Creates and returns a new {@link FoldingState}.
 	 * @param stateDesc The {@code StateDescriptor} that contains the name of the state.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
index ad9d417..b901d03 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java
@@ -49,13 +49,19 @@ import static java.util.Objects.requireNonNull;
 @PublicEvolving
 public abstract class StateDescriptor<S extends State, T> implements Serializable {
 
-	// Do not change the order of the elements in this enum, ordinal is used in serialization
+	/**
+	 * An enumeration of the types of supported states. Used to identify the state type
+	 * when writing and restoring checkpoints and savepoints.
+	 */
+	// IMPORTANT: Do not change the order of the elements in this enum, ordinal is used in serialization
 	public enum Type {
-		@Deprecated UNKNOWN, VALUE, LIST, REDUCING, FOLDING
+		@Deprecated UNKNOWN, VALUE, LIST, REDUCING, FOLDING, AGGREGATING
 	}
 
 	private static final long serialVersionUID = 1L;
 
+	// ------------------------------------------------------------------------
+
 	/** Name that uniquely identifies state created from this StateDescriptor. */
 	protected final String name;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index a2664f9..2b9eed9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -23,6 +23,7 @@ import org.apache.commons.lang3.ClassUtils;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.Public;
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.CrossFunction;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
@@ -189,6 +190,28 @@ public class TypeExtractor {
 	}
 
 	@PublicEvolving
+	public static <IN, ACC> TypeInformation<ACC> getAggregateFunctionAccumulatorType(
+			AggregateFunction<IN, ACC, ?> function,
+			TypeInformation<IN> inType,
+			String functionName,
+			boolean allowMissing)
+	{
+		return getUnaryOperatorReturnType(
+			function, AggregateFunction.class, 0, 1, inType, functionName, allowMissing);
+	}
+
+	@PublicEvolving
+	public static <IN, OUT> TypeInformation<OUT> getAggregateFunctionReturnType(
+			AggregateFunction<IN, ?, OUT> function,
+			TypeInformation<IN> inType,
+			String functionName,
+			boolean allowMissing)
+	{
+		return getUnaryOperatorReturnType(
+				function, AggregateFunction.class, 0, 2, inType, functionName, allowMissing);
+	}
+
+	@PublicEvolving
 	public static <IN, OUT> TypeInformation<OUT> getMapPartitionReturnTypes(MapPartitionFunction<IN, OUT> mapPartitionInterface, TypeInformation<IN> inType) {
 		return getMapPartitionReturnTypes(mapPartitionInterface, inType, null, false);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java b/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java
index 81ba7ab..0d4ea0c 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/CloseableRegistry.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.fs;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.AbstractCloseableRegistry;
 
 import java.io.Closeable;
@@ -32,6 +33,7 @@ import java.util.Map;
  * <p>
  * All methods in this class are thread-safe.
  */
+@Internal
 public class CloseableRegistry extends AbstractCloseableRegistry<Closeable, Object> {
 
 	private static final Object DUMMY = new Object();

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java
index 46b82c7..dd381a4 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.core.memory;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -26,6 +27,7 @@ import java.io.InputStream;
 /**
  * Un-synchronized stream similar to Java's ByteArrayInputStream that also exposes the current position.
  */
+@Internal
 public class ByteArrayInputStreamWithPos extends InputStream {
 
 	protected byte[] buffer;

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/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 fcca77a..c8e0d0d 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
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
 import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListState;
@@ -33,6 +35,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
 import org.apache.flink.runtime.state.internal.InternalFoldingState;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.internal.InternalListState;
@@ -159,6 +162,19 @@ public abstract class AbstractKeyedStateBackend<K>
 			ReducingStateDescriptor<T> stateDesc) throws Exception;
 
 	/**
+	 * Creates and returns a new {@link AggregatingState}.
+	 *
+	 * @param namespaceSerializer TypeSerializer for the state namespace.
+	 * @param stateDesc The {@code StateDescriptor} that contains the name of the state.
+	 *
+	 * @param <N> The type of the namespace.
+	 * @param <T> The type of the values that the {@code ListState} can store.
+	 */
+	protected abstract <N, T, ACC, R> InternalAggregatingState<N, T, R> createAggregatingState(
+			TypeSerializer<N> namespaceSerializer,
+			AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception;
+
+	/**
 	 * Creates and returns a new {@link FoldingState}.
 	 *
 	 * @param namespaceSerializer TypeSerializer for the state namespace.
@@ -265,6 +281,13 @@ public abstract class AbstractKeyedStateBackend<K>
 			}
 
 			@Override
+			public <T, ACC, R> AggregatingState<T, R> createAggregatingState(
+					AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
+				return AbstractKeyedStateBackend.this.createAggregatingState(namespaceSerializer, stateDesc);
+			}
+			
+
+			@Override
 			public <T, ACC> FoldingState<T, ACC> createFoldingState(FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
 				return AbstractKeyedStateBackend.this.createFoldingState(namespaceSerializer, stateDesc);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
new file mode 100644
index 0000000..624b83e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
@@ -0,0 +1,147 @@
+/*
+ * 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.heap;
+
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Heap-backed partitioned {@link ReducingState} that is
+ * snapshotted into files.
+ * 
+ * @param <K> The type of the key.
+ * @param <N> The type of the namespace.
+ * @param <IN> The type of the value added to the state.
+ * @param <ACC> The type of the value stored in the state (the accumulator type).
+ * @param <OUT> The type of the value returned from the state.
+ */
+public class HeapAggregatingState<K, N, IN, ACC, OUT>
+		extends AbstractHeapMergingState<K, N, IN, OUT, ACC, AggregatingState<IN, OUT>, AggregatingStateDescriptor<IN, ACC, OUT>>
+		implements InternalAggregatingState<N, IN, OUT> {
+
+	private final AggregateFunction<IN, ACC, OUT> aggFunction;
+
+	/**
+	 * Creates a new key/value state for the given hash map of key/value pairs.
+	 *
+	 * @param backend
+	 *             The state backend backing that created this state.
+	 * @param stateDesc
+	 *             The state identifier for the state. This contains name and can create a default state value.
+	 * @param stateTable
+	 *             The state table to use in this kev/value state. May contain initial state.
+	 * @param namespaceSerializer
+	 *             The serializer for the type that indicates the namespace
+	 */
+	public HeapAggregatingState(
+			KeyedStateBackend<K> backend,
+			AggregatingStateDescriptor<IN, ACC, OUT> stateDesc,
+			StateTable<K, N, ACC> stateTable,
+			TypeSerializer<K> keySerializer,
+			TypeSerializer<N> namespaceSerializer) {
+
+		super(backend, stateDesc, stateTable, keySerializer, namespaceSerializer);
+		this.aggFunction = stateDesc.getAggregateFunction();
+	}
+
+	// ------------------------------------------------------------------------
+	//  state access
+	// ------------------------------------------------------------------------
+
+	@Override
+	public OUT get() {
+		final K key = backend.getCurrentKey();
+
+		checkState(currentNamespace != null, "No namespace set.");
+		checkState(key != null, "No key set.");
+
+		Map<N, Map<K, ACC>> namespaceMap =
+				stateTable.get(backend.getCurrentKeyGroupIndex());
+
+		if (namespaceMap == null) {
+			return null;
+		}
+
+		Map<K, ACC> keyedMap = namespaceMap.get(currentNamespace);
+
+		if (keyedMap == null) {
+			return null;
+		}
+
+		ACC accumulator = keyedMap.get(key);
+		return aggFunction.getResult(accumulator);
+	}
+
+	@Override
+	public void add(IN value) throws IOException {
+		final K key = backend.getCurrentKey();
+
+		checkState(currentNamespace != null, "No namespace set.");
+		checkState(key != null, "No key set.");
+
+		if (value == null) {
+			clear();
+			return;
+		}
+
+		Map<N, Map<K, ACC>> namespaceMap =
+				stateTable.get(backend.getCurrentKeyGroupIndex());
+
+		if (namespaceMap == null) {
+			namespaceMap = createNewMap();
+			stateTable.set(backend.getCurrentKeyGroupIndex(), namespaceMap);
+		}
+
+		Map<K, ACC> keyedMap = namespaceMap.get(currentNamespace);
+
+		if (keyedMap == null) {
+			keyedMap = createNewMap();
+			namespaceMap.put(currentNamespace, keyedMap);
+		}
+
+		// if this is the first value for the key, create a new accumulator
+		ACC accumulator = keyedMap.get(key);
+		if (accumulator == null) {
+			accumulator = aggFunction.createAccumulator();
+			keyedMap.put(key, accumulator);
+		}
+
+		// 
+		aggFunction.add(value, accumulator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  state merging
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected ACC mergeState(ACC a, ACC b) throws Exception {
+		return aggFunction.merge(a, b);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/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 b05b874..b4c2b8b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -19,6 +19,8 @@
 package org.apache.flink.runtime.state.heap;
 
 import org.apache.commons.io.IOUtils;
+
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
@@ -50,12 +52,14 @@ import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
 import org.apache.flink.runtime.state.internal.InternalFoldingState;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalReducingState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -169,6 +173,15 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
+	public <N, T, ACC, R> InternalAggregatingState<N, T, R> createAggregatingState(
+			TypeSerializer<N> namespaceSerializer,
+			AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
+
+		StateTable<K, N, ACC> stateTable = tryRegisterStateTable(namespaceSerializer, stateDesc);
+		return new HeapAggregatingState<>(this, stateDesc, stateTable, keySerializer, namespaceSerializer);
+	}
+
+	@Override
 	protected <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
 			TypeSerializer<N> namespaceSerializer,
 			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
index 7804cb4..090a660 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java
@@ -49,7 +49,7 @@ public class HeapReducingState<K, N, V>
 	 * @param backend The state backend backing that created this state.
 	 * @param stateDesc The state identifier for the state. This contains name
 	 *                           and can create a default state value.
-	 * @param stateTable The state tab;e to use in this kev/value state. May contain initial state.
+	 * @param stateTable The state table to use in this kev/value state. May contain initial state.
 	 */
 	public HeapReducingState(
 			KeyedStateBackend<K> backend,

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAggregatingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAggregatingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAggregatingState.java
new file mode 100644
index 0000000..15a8e31
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalAggregatingState.java
@@ -0,0 +1,33 @@
+/*
+ * 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.internal;
+
+import org.apache.flink.api.common.state.AggregatingState;
+
+/**
+ * The peer to the {@link AggregatingState} in the internal state type hierarchy.
+ * 
+ * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
+ * 
+ * @param <N>   The type of the namespace
+ * @param <IN>  Type of the value added to the state.
+ * @param <OUT> Type of the value extracted from the state.
+ */
+public interface InternalAggregatingState<N, IN, OUT> 
+		extends InternalMergingState<N, IN, OUT>, AggregatingState<IN, OUT> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
index 40e625c..76fa58f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/internal/InternalReducingState.java
@@ -26,6 +26,6 @@ import org.apache.flink.api.common.state.ReducingState;
  * <p>See {@link InternalKvState} for a description of the internal state hierarchy.
  * 
  * @param <N> The type of the namespace
- * @param <T> The type of elements in the list
+ * @param <T> The type of elements in the aggregated by the ReduceFunction
  */
 public interface InternalReducingState<N, T> extends InternalMergingState<N, T, T>, ReducingState<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
index 2448540..66e8d02 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
@@ -161,12 +161,13 @@ public class SerializationProxiesTest {
 	@Test
 	public void testFixTypeOrder() {
 		// ensure all elements are covered
-		Assert.assertEquals(5, StateDescriptor.Type.values().length);
+		Assert.assertEquals(6, StateDescriptor.Type.values().length);
 		// fix the order of elements to keep serialization format stable
 		Assert.assertEquals(0, StateDescriptor.Type.UNKNOWN.ordinal());
 		Assert.assertEquals(1, StateDescriptor.Type.VALUE.ordinal());
 		Assert.assertEquals(2, StateDescriptor.Type.LIST.ordinal());
 		Assert.assertEquals(3, StateDescriptor.Type.REDUCING.ordinal());
 		Assert.assertEquals(4, StateDescriptor.Type.FOLDING.ordinal());
+		Assert.assertEquals(5, StateDescriptor.Type.AGGREGATING.ordinal());
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapAggregatingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapAggregatingStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapAggregatingStateTest.java
new file mode 100644
index 0000000..a7ae5be
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapAggregatingStateTest.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.heap;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalAggregatingState;
+
+import org.junit.Test;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the simple Java heap objects implementation of the {@link AggregatingState}.
+ */
+public class HeapAggregatingStateTest {
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final AggregatingStateDescriptor<Long, MutableLong, Long> stateDescr =
+				new AggregatingStateDescriptor<>("my-state", new AddingFunction(), MutableLong.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			InternalAggregatingState<VoidNamespace, Long, Long> state =
+					keyedBackend.createAggregatingState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(28L, state.get().longValue());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(28L, state.get().longValue());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(9L, state.get().longValue());
+			state.clear();
+
+			// make sure all lists / maps are cleared
+
+			StateTable<String, VoidNamespace, MutableLong> stateTable =
+					((HeapAggregatingState<String, VoidNamespace, Long, MutableLong, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final AggregatingStateDescriptor<Long, MutableLong, Long> stateDescr =
+				new AggregatingStateDescriptor<>("my-state", new AddingFunction(), MutableLong.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final Integer namespace1 = 1;
+		final Integer namespace2 = 2;
+		final Integer namespace3 = 3;
+
+		final Long expectedResult = 165L;
+
+		final HeapKeyedStateBackend<String> keyedBackend = createKeyedBackend();
+
+		try {
+			InternalAggregatingState<Integer, Long, Long> state =
+					keyedBackend.createAggregatingState(IntSerializer.INSTANCE, stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(namespace2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(namespace3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(namespace1, asList(namespace2, namespace3));
+			state.setCurrentNamespace(namespace1);
+			assertEquals(expectedResult, state.get());
+
+			// make sure all lists / maps are cleared
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("ghi");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(namespace1);
+			state.clear();
+
+			StateTable<String, Integer, MutableLong> stateTable =
+					((HeapAggregatingState<String, Integer, Long, MutableLong, Long>) state).stateTable;
+
+			assertTrue(stateTable.isEmpty());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static HeapKeyedStateBackend<String> createKeyedBackend() throws Exception {
+		return new HeapKeyedStateBackend<>(
+				mock(TaskKvStateRegistry.class),
+				StringSerializer.INSTANCE,
+				HeapAggregatingStateTest.class.getClassLoader(),
+				16,
+				new KeyGroupRange(0, 15));
+	}
+
+	// ------------------------------------------------------------------------
+	//  test functions
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static class AddingFunction implements AggregateFunction<Long, MutableLong, Long> {
+
+		@Override
+		public MutableLong createAccumulator() {
+			return new MutableLong();
+		}
+
+		@Override
+		public void add(Long value, MutableLong accumulator) {
+			accumulator.value += value;
+		}
+
+		@Override
+		public Long getResult(MutableLong accumulator) {
+			return accumulator.value;
+		}
+
+		@Override
+		public MutableLong merge(MutableLong a, MutableLong b) {
+			a.value += b.value;
+			return a;
+		}
+	}
+
+	private static final class MutableLong {
+		long value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 704875b..30e64c4 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -20,10 +20,12 @@ package org.apache.flink.streaming.api.datastream;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFunction;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
@@ -36,6 +38,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
 import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
 import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.windowing.AggregateApplyWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction;
@@ -63,6 +66,8 @@ import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * A {@code WindowedStream} represents a data stream where elements are grouped by
  * key, and for each key, the stream of elements is split into windows based on a
@@ -356,6 +361,10 @@ public class WindowedStream<T, K, W extends Window> {
 		return input.transform(opName, resultType, operator);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Fold Function
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Applies the given fold function to each window. The window function is called for each
 	 * evaluation of the window for each key individually. The output of the reduce function is
@@ -500,6 +509,172 @@ public class WindowedStream<T, K, W extends Window> {
 		return input.transform(opName, resultType, operator);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Aggregation Function
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies the given fold function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the reduce function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <ACC, R> SingleOutputStreamOperator<R> aggregate(AggregateFunction<T, ACC, R> function) {
+		checkNotNull(function, "function");
+
+		if (function instanceof RichFunction) {
+			throw new UnsupportedOperationException("This aggregation function cannot be a RichFunction.");
+		}
+
+		TypeInformation<ACC> accumulatorType = TypeExtractor.getAggregateFunctionAccumulatorType(
+				function, input.getType(), null, false);
+
+		TypeInformation<R> resultType = TypeExtractor.getAggregateFunctionReturnType(
+				function, input.getType(), null, false);
+
+		return aggregate(function, accumulatorType, resultType);
+	}
+
+	/**
+	 * Applies the given aggregation function to each window. The aggregation function is called for
+	 * each element, aggregating values incrementally and keeping the state to one accumulator
+	 * per key and window.
+	 *
+	 * @param function The aggregation function.
+	 * @return The data stream that is the result of applying the aggregation function to the window.
+	 */
+	public <ACC, R> SingleOutputStreamOperator<R> aggregate(
+			AggregateFunction<T, ACC, R> function,
+			TypeInformation<ACC> accumulatorType,
+			TypeInformation<R> resultType) {
+
+		checkNotNull(function, "function");
+		checkNotNull(accumulatorType, "accumulatorType");
+		checkNotNull(resultType, "resultType");
+
+		if (function instanceof RichFunction) {
+			throw new UnsupportedOperationException("This aggregation function cannot be a RichFunction.");
+		}
+
+		return aggregate(function, new PassThroughWindowFunction<K, W, R>(), accumulatorType, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>Arriving data is incrementally aggregated using the given aggregate function. This means
+	 * that the window function typically has only a single value to process when called.
+	 *
+	 * @param aggFunction The aggregate function that is used for incremental aggregation.
+	 * @param windowFunction The window function.
+	 * 
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <ACC, R> SingleOutputStreamOperator<R> aggregate(
+			AggregateFunction<T, ACC, R> aggFunction,
+			WindowFunction<R, R, K, W> windowFunction) {
+
+		checkNotNull(aggFunction, "aggFunction");
+		checkNotNull(windowFunction, "windowFunction");
+
+		TypeInformation<ACC> accumulatorType = TypeExtractor.getAggregateFunctionAccumulatorType(
+				aggFunction, input.getType(), null, false);
+
+		TypeInformation<R> resultType = TypeExtractor.getAggregateFunctionReturnType(
+				aggFunction, input.getType(), null, false);
+
+		return aggregate(aggFunction, windowFunction, accumulatorType, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>Arriving data is incrementally aggregated using the given aggregate function. This means
+	 * that the window function typically has only a single value to process when called.
+	 *
+	 * @param aggregateFunction The aggregation function that is used for incremental aggregation.
+	 * @param windowFunction The window function.
+	 * @param accumulatorType Type information for the internal accumulator type of the aggregation function
+	 * @param resultType Type information for the result type of the window function
+	 *    
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <ACC, R> SingleOutputStreamOperator<R> aggregate(
+			AggregateFunction<T, ACC, R> aggregateFunction,
+			WindowFunction<R, R, K, W> windowFunction, 
+			TypeInformation<ACC> accumulatorType,
+			TypeInformation<R> resultType) {
+
+		checkNotNull(aggregateFunction, "aggregateFunction");
+		checkNotNull(windowFunction, "windowFunction");
+		checkNotNull(accumulatorType, "accumulatorType");
+		checkNotNull(resultType, "resultType");
+
+		if (aggregateFunction instanceof RichFunction) {
+			throw new UnsupportedOperationException("This aggregate function cannot be a RichFunction.");
+		}
+
+		//clean the closures
+		windowFunction = input.getExecutionEnvironment().clean(windowFunction);
+		aggregateFunction = input.getExecutionEnvironment().clean(aggregateFunction);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "WindowedStream." + callLocation;
+
+		String opName;
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, R> operator;
+
+		if (evictor != null) {
+			@SuppressWarnings({"unchecked", "rawtypes"})
+			TypeSerializer<StreamRecord<T>> streamRecordSerializer =
+					(TypeSerializer<StreamRecord<T>>) new StreamElementSerializer(input.getType().createSerializer(getExecutionEnvironment().getConfig()));
+
+			ListStateDescriptor<StreamRecord<T>> stateDesc =
+					new ListStateDescriptor<>("window-contents", streamRecordSerializer);
+
+			opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + evictor + ", " + udfName + ")";
+
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					stateDesc,
+					new InternalIterableWindowFunction<>(new AggregateApplyWindowFunction<>(aggregateFunction, windowFunction)),
+					trigger,
+					evictor,
+					allowedLateness);
+
+		} else {
+			AggregatingStateDescriptor<T, ACC, R> stateDesc = new AggregatingStateDescriptor<>("window-contents",
+					aggregateFunction, accumulatorType.createSerializer(getExecutionEnvironment().getConfig()));
+
+			opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")";
+
+			operator = new WindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					stateDesc,
+					new InternalSingleValueWindowFunction<>(windowFunction),
+					trigger,
+					allowedLateness);
+		}
+
+		return input.transform(opName, resultType, operator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Window Function (apply)
+	// ------------------------------------------------------------------------
+	
 	/**
 	 * Applies the given window function to each window. The window function is called for each
 	 * evaluation of the window for each key individually. The output of the window function is
@@ -792,7 +967,7 @@ public class WindowedStream<T, K, W extends Window> {
 	}
 
 	// ------------------------------------------------------------------------
-	//  Aggregations on the keyed windows
+	//  Pre-defined aggregations on the keyed windows
 	// ------------------------------------------------------------------------
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java
new file mode 100644
index 0000000..1b9fa88
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyAllWindowFunction.java
@@ -0,0 +1,56 @@
+/**
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.util.Collections;
+
+@Internal
+public class AggregateApplyAllWindowFunction<W extends Window, T, ACC, R>
+	extends WrappingFunction<AllWindowFunction<R, R, W>>
+	implements AllWindowFunction<T, R, W> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final AggregateFunction<T, ACC, R> aggFunction;
+
+	public AggregateApplyAllWindowFunction(
+			AggregateFunction<T, ACC, R> aggFunction,
+			AllWindowFunction<R, R, W> windowFunction) {
+
+		super(windowFunction);
+		this.aggFunction = aggFunction;
+	}
+
+	@Override
+	public void apply(W window, Iterable<T> values, Collector<R> out) throws Exception {
+		final ACC acc = aggFunction.createAccumulator();
+
+		for (T value : values) {
+			aggFunction.add(value, acc);
+		}
+
+		wrappedFunction.apply(window, Collections.singletonList(aggFunction.getResult(acc)), out);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/09380e49/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.java
new file mode 100644
index 0000000..5200bc2
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/AggregateApplyWindowFunction.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.streaming.api.functions.windowing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.util.Collections;
+
+@Internal
+public class AggregateApplyWindowFunction<K, W extends Window, T, ACC, R>
+	extends WrappingFunction<WindowFunction<R, R, K, W>>
+	implements WindowFunction<T, R, K, W> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final AggregateFunction<T, ACC, R> aggFunction;
+
+	public AggregateApplyWindowFunction(AggregateFunction<T, ACC, R> aggFunction, WindowFunction<R, R, K, W> windowFunction) {
+		super(windowFunction);
+		this.aggFunction = aggFunction;
+	}
+
+	@Override
+	public void apply(K key, W window, Iterable<T> values, Collector<R> out) throws Exception {
+		final ACC acc = aggFunction.createAccumulator();
+
+		for (T val : values) {
+			aggFunction.add(val, acc);
+		}
+
+		wrappedFunction.apply(key, window, Collections.singletonList(aggFunction.getResult(acc)), out);
+	}
+}


[5/9] flink git commit: [FLINK-5590] [runtime] Add proper internal state hierarchy

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/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 628d663..9b8af58 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
@@ -19,10 +19,10 @@
 package org.apache.flink.streaming.runtime.operators.windowing;
 
 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.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.State;
@@ -42,6 +42,9 @@ import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.state.ArrayListSerializer;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalAppendingState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.runtime.state.internal.InternalMergingState;
 import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType;
 import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
@@ -107,22 +110,16 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 
 	protected final WindowAssigner<? super IN, W> windowAssigner;
 
-	protected final KeySelector<IN, K> keySelector;
-
-	protected final Trigger<? super IN, ? super W> trigger;
+	private final KeySelector<IN, K> keySelector;
 
-	protected final StateDescriptor<? extends AppendingState<IN, ACC>, ?> windowStateDescriptor;
+	private final Trigger<? super IN, ? super W> trigger;
 
-	protected final ListStateDescriptor<Tuple2<W, W>> mergingWindowsDescriptor;
+	private final StateDescriptor<? extends AppendingState<IN, ACC>, ?> windowStateDescriptor;
 
-	/**
-	 * For serializing the key in checkpoints.
-	 */
+	/** For serializing the key in checkpoints. */
 	protected final TypeSerializer<K> keySerializer;
 
-	/**
-	 * For serializing the window in checkpoints.
-	 */
+	/** For serializing the window in checkpoints. */
 	protected final TypeSerializer<W> windowSerializer;
 
 	/**
@@ -133,15 +130,23 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 *         {@code window.maxTimestamp + allowedLateness} landmark.
 	 * </ul>
 	 */
-	protected final long allowedLateness;
+	private final long allowedLateness;
 
 	// ------------------------------------------------------------------------
 	// State that is not checkpointed
 	// ------------------------------------------------------------------------
 
-	/**
-	 * This is given to the {@code InternalWindowFunction} for emitting elements with a given timestamp.
-	 */
+	/** The state in which the window contents is stored. Each window is a namespace */
+	private transient InternalAppendingState<W, IN, ACC> windowState;
+
+	/** The {@link #windowState}, typed to merging state for merging windows.
+	 * Null if the window state is not mergeable */
+	private transient InternalMergingState<W, IN, ACC> windowMergingState;
+
+	/** The state that holds the merging window metadata (the sets that describe what is merged) */
+	private transient InternalListState<VoidNamespace, Tuple2<W, W>> mergingSetsState;
+
+	/** 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);
@@ -234,14 +239,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		this.allowedLateness = allowedLateness;
 		this.legacyWindowOperatorType = legacyWindowOperatorType;
 
-		if (windowAssigner instanceof MergingWindowAssigner) {
-			@SuppressWarnings({"unchecked", "rawtypes"})
-			TupleSerializer<Tuple2<W, W>> tupleSerializer = new TupleSerializer<>((Class) Tuple2.class, new TypeSerializer[] {windowSerializer, windowSerializer} );
-			mergingWindowsDescriptor = new ListStateDescriptor<>("merging-window-set", tupleSerializer);
-		} else {
-			mergingWindowsDescriptor = null;
-		}
-
 		setChainingStrategy(ChainingStrategy.ALWAYS);
 	}
 
@@ -263,6 +260,43 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			}
 		};
 
+		// create (or restore) the state that hold the actual window contents
+		// NOTE - the state may be null in the case of the overriding evicting window operator
+		if (windowStateDescriptor != null) {
+			windowState = (InternalAppendingState<W, IN, ACC>) getOrCreateKeyedState(windowSerializer, windowStateDescriptor);
+		}
+
+		// create the typed and helper states for merging windows
+		if (windowAssigner instanceof MergingWindowAssigner) {
+
+			// store a typed reference for the state of merging windows - sanity check
+			if (windowState instanceof InternalMergingState) {
+				windowMergingState = (InternalMergingState<W, IN, ACC>) windowState;
+			}
+			// TODO this sanity check should be here, but is prevented by an incorrect test (pending validation)
+			// TODO see WindowOperatorTest.testCleanupTimerWithEmptyFoldingStateForSessionWindows()
+			// TODO activate the sanity check once resolved
+//			else if (windowState != null) {
+//				throw new IllegalStateException(
+//						"The window uses a merging assigner, but the window state is not mergeable.");
+//			}
+
+			@SuppressWarnings("unchecked")
+			final Class<Tuple2<W, W>> typedTuple = (Class<Tuple2<W, W>>) (Class<?>) Tuple2.class;
+
+			final TupleSerializer<Tuple2<W, W>> tupleSerializer = new TupleSerializer<>(
+					typedTuple,
+					new TypeSerializer[] {windowSerializer, windowSerializer} );
+
+			final ListStateDescriptor<Tuple2<W, W>> mergingSetsStateDescriptor =
+					new ListStateDescriptor<>("merging-window-set", tupleSerializer);
+
+			// get the state that stores the merging sets
+			mergingSetsState = (InternalListState<VoidNamespace, Tuple2<W, W>>) 
+					getOrCreateKeyedState(VoidNamespaceSerializer.INSTANCE, mergingSetsStateDescriptor);
+			mergingSetsState.setCurrentNamespace(VoidNamespace.INSTANCE);
+		}
+
 		registerRestoredLegacyStateState();
 	}
 
@@ -283,12 +317,11 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
 	public void processElement(StreamRecord<IN> element) throws Exception {
-		Collection<W> elementWindows = windowAssigner.assignWindows(
+		final Collection<W> elementWindows = windowAssigner.assignWindows(
 			element.getValue(), element.getTimestamp(), windowAssignerContext);
-
-		final K key = (K) getKeyedStateBackend().getCurrentKey();
+		
+		final K key = this.<K>getKeyedStateBackend().getCurrentKey();
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			MergingWindowSet<W> mergingWindows = getMergingWindowSet();
@@ -315,11 +348,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 						}
 
 						// merge the merged state windows into the newly resulting state window
-						getKeyedStateBackend().mergePartitionedStates(
-							stateWindowResult,
-							mergedStateWindows,
-							windowSerializer,
-							(StateDescriptor<? extends MergingState<?,?>, ?>) windowStateDescriptor);
+						windowMergingState.mergeNamespaces(stateWindowResult, mergedStateWindows);
 					}
 				});
 
@@ -334,8 +363,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					throw new IllegalStateException("Window " + window + " is not in in-flight window set.");
 				}
 
-				AppendingState<IN, ACC> windowState =
-						getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+				windowState.setCurrentNamespace(stateWindow);
 				windowState.add(element.getValue());
 
 				context.key = key;
@@ -368,8 +396,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					continue;
 				}
 
-				AppendingState<IN, ACC> windowState =
-						getPartitionedState(window, windowSerializer, windowStateDescriptor);
+				windowState.setCurrentNamespace(window);
 				windowState.add(element.getValue());
 
 				context.key = key;
@@ -399,8 +426,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		context.key = timer.getKey();
 		context.window = timer.getNamespace();
 
-		AppendingState<IN, ACC> windowState;
-		MergingWindowSet<W> mergingWindows = null;
+		MergingWindowSet<W> mergingWindows;
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
@@ -411,12 +437,11 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				// so it is safe to just ignore
 				return;
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			
+			windowState.setCurrentNamespace(stateWindow);
 		} else {
-			windowState = getPartitionedState(
-					context.window,
-					windowSerializer,
-					windowStateDescriptor);
+			windowState.setCurrentNamespace(context.window);
+			mergingWindows = null;
 		}
 
 		ACC contents = windowState.get();
@@ -440,8 +465,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		context.key = timer.getKey();
 		context.window = timer.getNamespace();
 
-		AppendingState<IN, ACC> windowState;
-		MergingWindowSet<W> mergingWindows = null;
+		MergingWindowSet<W> mergingWindows;
 
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindows = getMergingWindowSet();
@@ -452,9 +476,10 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 				// so it is safe to just ignore
 				return;
 			}
-			windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			windowState.setCurrentNamespace(stateWindow);
 		} else {
-			windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
+			windowState.setCurrentNamespace(context.window);
+			mergingWindows = null;
 		}
 
 		ACC contents = windowState.get();
@@ -507,13 +532,9 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 * {@link MergingWindowSet#persist()}.
 	 */
 	protected MergingWindowSet<W> getMergingWindowSet() throws Exception {
-		ListState<Tuple2<W, W>> mergeState =
-				getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, mergingWindowsDescriptor);
-
-		@SuppressWarnings({"unchecked", "rawtypes"})
-		MergingWindowSet<W> mergingWindows = new MergingWindowSet<>((MergingWindowAssigner) windowAssigner, mergeState);
-
-		return mergingWindows;
+		@SuppressWarnings("unchecked")
+		MergingWindowAssigner<? super IN, W> mergingAssigner = (MergingWindowAssigner<? super IN, W>) windowAssigner;
+		return new MergingWindowSet<>(mergingAssigner, mergingSetsState);
 	}
 
 	/**
@@ -655,11 +676,19 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		public <S extends MergingState<?, ?>> void mergePartitionedState(StateDescriptor<S, ?> stateDescriptor) {
 			if (mergedWindows != null && mergedWindows.size() > 0) {
 				try {
-					WindowOperator.this.getKeyedStateBackend().mergePartitionedStates(window,
-							mergedWindows,
-							windowSerializer,
-							stateDescriptor);
-				} catch (Exception e) {
+					S rawState = getKeyedStateBackend().getOrCreateKeyedState(windowSerializer, stateDescriptor);
+
+					if (rawState instanceof InternalMergingState) {
+						@SuppressWarnings("unchecked")
+						InternalMergingState<W, ?, ?> mergingState = (InternalMergingState<W, ?, ?>) rawState;
+						mergingState.mergeNamespaces(window, mergedWindows);
+					}
+					else {
+						throw new IllegalArgumentException(
+								"The given state descriptor does not refer to a mergeable state (MergingState)");
+					}
+				}
+				catch (Exception e) {
 					throw new RuntimeException("Error while merging state.", e);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index 0e2d1e8..2faa506 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -72,6 +72,7 @@ import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.Serializable;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -83,6 +84,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+@SuppressWarnings("serial")
 public class WindowOperatorTest extends TestLogger {
 
 	// For counting if close() is called the correct number of times on the SumReducer
@@ -758,7 +760,7 @@ public class WindowOperatorTest extends TestLogger {
 				0);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -846,7 +848,7 @@ public class WindowOperatorTest extends TestLogger {
 				0);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
 
@@ -1124,7 +1126,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 		
 		testHarness.open();
 		
@@ -1184,7 +1186,7 @@ public class WindowOperatorTest extends TestLogger {
 					LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 		
 		testHarness.open();
 
@@ -1250,7 +1252,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -1310,7 +1312,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 		
@@ -1385,7 +1387,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 		
@@ -1475,7 +1477,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 		
@@ -1559,7 +1561,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -1643,7 +1645,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 		
@@ -1736,7 +1738,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 		
@@ -1821,7 +1823,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -1902,11 +1904,11 @@ public class WindowOperatorTest extends TestLogger {
 				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
 				windowStateDesc,
 				new InternalIterableWindowFunction<>(new PassThroughFunction2()),
-				new EventTimeTriggerAccumGC(LATENESS),
+					new EventTimeTriggerAccumGC(LATENESS),
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, String> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -1929,7 +1931,7 @@ public class WindowOperatorTest extends TestLogger {
 		testHarness.close();
 	}
 
-	private class PassThroughFunction2 implements WindowFunction<Tuple2<String, Integer>, String, String, TimeWindow> {
+	private static class PassThroughFunction2 implements WindowFunction<Tuple2<String, Integer>, String, String, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -1960,7 +1962,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2006,7 +2008,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2063,7 +2065,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2108,7 +2110,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2152,7 +2154,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2172,6 +2174,7 @@ public class WindowOperatorTest extends TestLogger {
 		testHarness.close();
 	}
 
+	// TODO this test seems invalid, as it uses the unsupported combination of merging windows and folding window state
 	@Test
 	public void testCleanupTimerWithEmptyFoldingStateForSessionWindows() throws Exception {
 		final int GAP_SIZE = 3;
@@ -2206,7 +2209,7 @@ public class WindowOperatorTest extends TestLogger {
 				LATENESS);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		testHarness.open();
 
@@ -2230,7 +2233,7 @@ public class WindowOperatorTest extends TestLogger {
 	//  UDFs
 	// ------------------------------------------------------------------------
 
-	private class PassThroughFunction implements WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow> {
+	private static class PassThroughFunction implements WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -2289,7 +2292,7 @@ public class WindowOperatorTest extends TestLogger {
 	}
 
 	@SuppressWarnings("unchecked")
-	private static class Tuple2ResultSortComparator implements Comparator<Object> {
+	private static class Tuple2ResultSortComparator implements Comparator<Object>, Serializable {
 		@Override
 		public int compare(Object o1, Object o2) {
 			if (o1 instanceof Watermark || o2 instanceof Watermark) {
@@ -2311,7 +2314,7 @@ public class WindowOperatorTest extends TestLogger {
 	}
 
 	@SuppressWarnings("unchecked")
-	private static class Tuple3ResultSortComparator implements Comparator<Object> {
+	private static class Tuple3ResultSortComparator implements Comparator<Object>, Serializable {
 		@Override
 		public int compare(Object o1, Object o2) {
 			if (o1 instanceof Watermark || o2 instanceof Watermark) {
@@ -2403,15 +2406,11 @@ public class WindowOperatorTest extends TestLogger {
 	 * purge the state of the fired window. This is to test the state
 	 * garbage collection mechanism.
 	 */
-	public class EventTimeTriggerAccumGC extends Trigger<Object, TimeWindow> {
+	public static class EventTimeTriggerAccumGC extends Trigger<Object, TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		private long cleanupTime;
 
-		private EventTimeTriggerAccumGC() {
-			cleanupTime = 0L;
-		}
-
 		public EventTimeTriggerAccumGC(long cleanupTime) {
 			this.cleanupTime = cleanupTime;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java b/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
index 0e1aca0..0562443 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
@@ -18,9 +18,7 @@
 
 package org.apache.flink.test.query;
 
-
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
@@ -30,10 +28,14 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializerTest;
 import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalListState;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 
@@ -59,14 +61,18 @@ public final class KVStateRequestSerializerRocksDBTest {
 	 */
 	final static class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {
 
-		RocksDBKeyedStateBackend2(final JobID jobId,
-			final String operatorIdentifier,
-			final ClassLoader userCodeClassLoader,
-			final File instanceBasePath, final DBOptions dbOptions,
-			final ColumnFamilyOptions columnFamilyOptions,
-			final TaskKvStateRegistry kvStateRegistry,
-			final TypeSerializer<K> keySerializer, final int numberOfKeyGroups,
-			final KeyGroupRange keyGroupRange) throws Exception {
+		RocksDBKeyedStateBackend2(
+				final JobID jobId,
+				final String operatorIdentifier,
+				final ClassLoader userCodeClassLoader,
+				final File instanceBasePath,
+				final DBOptions dbOptions,
+				final ColumnFamilyOptions columnFamilyOptions,
+				final TaskKvStateRegistry kvStateRegistry,
+				final TypeSerializer<K> keySerializer,
+				final int numberOfKeyGroups,
+				final KeyGroupRange keyGroupRange) throws Exception {
+
 			super(jobId, operatorIdentifier, userCodeClassLoader,
 				instanceBasePath,
 				dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer,
@@ -74,9 +80,10 @@ public final class KVStateRequestSerializerRocksDBTest {
 		}
 
 		@Override
-		public <N, T> ListState<T> createListState(
+		public <N, T> InternalListState<N, T> createListState(
 			final TypeSerializer<N> namespaceSerializer,
 			final ListStateDescriptor<T> stateDesc) throws Exception {
+
 			return super.createListState(namespaceSerializer, stateDesc);
 		}
 	}
@@ -90,8 +97,7 @@ public final class KVStateRequestSerializerRocksDBTest {
 	 */
 	@Test
 	public void testListSerialization() throws Exception {
-		final long key = 0l;
-		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
+		final long key = 0L;
 
 		// objects for RocksDB state list serialisation
 		DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
@@ -110,9 +116,10 @@ public final class KVStateRequestSerializerRocksDBTest {
 			);
 		longHeapKeyedStateBackend.setCurrentKey(key);
 
-		final ListState<Long> listState = longHeapKeyedStateBackend
+		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend
 			.createListState(VoidNamespaceSerializer.INSTANCE,
 				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
+
 		KvStateRequestSerializerTest.testListSerialization(key, listState);
 	}
 }


[7/9] flink git commit: [FLINK-5590] [runtime] Add proper internal state hierarchy

Posted by se...@apache.org.
[FLINK-5590] [runtime] Add proper internal state hierarchy

This introduces an internal state hierarchy that mirrors the external state hierarchy,
but gives the runtime access to methods that should not be part of the user facing API,
such as:
  - setting namespaces
  - accessing raw values
  - merging namespaces


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

Branch: refs/heads/master
Commit: 3b97128f05bacfb80afe4a2a49741c31ff306cd2
Parents: 2f1c474
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jan 13 15:17:09 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:53:39 2017 +0100

----------------------------------------------------------------------
 .../streaming/state/AbstractRocksDBState.java   |   9 +-
 .../streaming/state/RocksDBFoldingState.java    |   3 +-
 .../state/RocksDBKeyedStateBackend.java         |  20 +-
 .../streaming/state/RocksDBListState.java       |  42 ++-
 .../streaming/state/RocksDBReducingState.java   |  74 +++++-
 .../streaming/state/RocksDBValueState.java      |   3 +-
 .../streaming/state/RocksDBListStateTest.java   | 232 +++++++++++++++++
 .../state/RocksDBReducingStateTest.java         | 235 +++++++++++++++++
 .../flink/runtime/execution/Environment.java    |   4 +-
 .../runtime/io/network/NetworkEnvironment.java  |   5 +-
 .../runtime/jobmaster/JobMasterGateway.java     |   6 +-
 .../apache/flink/runtime/query/KvStateID.java   |   4 +-
 .../flink/runtime/query/KvStateLocation.java    |   4 +-
 .../runtime/query/KvStateLocationRegistry.java  |   4 +-
 .../flink/runtime/query/KvStateMessage.java     |  10 +-
 .../flink/runtime/query/KvStateRegistry.java    |  10 +-
 .../runtime/query/TaskKvStateRegistry.java      |   4 +-
 .../query/netty/KvStateServerHandler.java       |  12 +-
 .../query/netty/message/KvStateRequest.java     |   4 +-
 .../state/AbstractKeyedStateBackend.java        | 182 +++++--------
 .../flink/runtime/state/KeyedStateBackend.java  |  35 ++-
 .../org/apache/flink/runtime/state/KvState.java |  52 ----
 .../flink/runtime/state/VoidNamespace.java      |  43 +++-
 .../state/heap/AbstractHeapMergingState.java    | 123 +++++++++
 .../runtime/state/heap/AbstractHeapState.java   |   4 +-
 .../runtime/state/heap/HeapFoldingState.java    |   8 +-
 .../state/heap/HeapKeyedStateBackend.java       |  35 ++-
 .../flink/runtime/state/heap/HeapListState.java |  19 +-
 .../runtime/state/heap/HeapReducingState.java   |  23 +-
 .../runtime/state/heap/HeapValueState.java      |   3 +-
 .../flink/runtime/state/heap/StateTable.java    |  18 ++
 .../state/internal/InternalAppendingState.java  |  32 +++
 .../state/internal/InternalFoldingState.java    |  32 +++
 .../runtime/state/internal/InternalKvState.java |  79 ++++++
 .../state/internal/InternalListState.java       |  31 +++
 .../state/internal/InternalMergingState.java    |  46 ++++
 .../state/internal/InternalReducingState.java   |  31 +++
 .../state/internal/InternalValueState.java      |  31 +++
 .../runtime/query/netty/KvStateClientTest.java  |   4 +-
 .../query/netty/KvStateServerHandlerTest.java   |   6 +-
 .../message/KvStateRequestSerializerTest.java   |  30 +--
 .../runtime/state/StateBackendTestBase.java     |  43 ++--
 .../runtime/state/heap/HeapListStateTest.java   | 248 ++++++++++++++++++
 .../state/heap/HeapReducingStateTest.java       | 254 +++++++++++++++++++
 .../api/operators/AbstractStreamOperator.java   |  23 +-
 .../api/windowing/windows/TimeWindow.java       |  38 ++-
 .../streaming/api/windowing/windows/Window.java |   5 +
 .../windowing/EvictingWindowOperator.java       | 101 ++++----
 .../operators/windowing/WindowOperator.java     | 149 ++++++-----
 .../operators/windowing/WindowOperatorTest.java |  57 ++---
 .../KVStateRequestSerializerRocksDBTest.java    |  35 ++-
 51 files changed, 2054 insertions(+), 451 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
index 6785f17..89f41aa 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
@@ -25,14 +25,13 @@ import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.util.Preconditions;
+
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
@@ -48,9 +47,7 @@ import java.io.IOException;
  * @param <SD> The type of {@link StateDescriptor}.
  */
 public abstract class AbstractRocksDBState<K, N, S extends State, SD extends StateDescriptor<S, V>, V>
-		implements KvState<N>, State {
-
-	private static final Logger LOG = LoggerFactory.getLogger(AbstractRocksDBState.class);
+		implements InternalKvState<N>, State {
 
 	/** Serializer for the namespace */
 	private final TypeSerializer<N> namespaceSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
index 9c2bf4f..26dc3dd 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.internal.InternalFoldingState;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
@@ -41,7 +42,7 @@ import java.io.IOException;
  */
 public class RocksDBFoldingState<K, N, T, ACC>
 	extends AbstractRocksDBState<K, N, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>, ACC>
-	implements FoldingState<T, ACC> {
+	implements InternalFoldingState<N, T, ACC> {
 
 	/** Serializer for the values */
 	private final TypeSerializer<ACC> valueSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/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 b207af6..5a6d102 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
@@ -18,14 +18,10 @@
 package org.apache.flink.contrib.streaming.state;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
@@ -52,6 +48,10 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
 import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.internal.InternalFoldingState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.InstantiationUtil;
@@ -812,7 +812,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	protected <N, T> ValueState<T> createValueState(TypeSerializer<N> namespaceSerializer,
+	protected <N, T> InternalValueState<N, T> createValueState(
+			TypeSerializer<N> namespaceSerializer,
 			ValueStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
@@ -821,7 +822,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	protected <N, T> ListState<T> createListState(TypeSerializer<N> namespaceSerializer,
+	protected <N, T> InternalListState<N, T> createListState(
+			TypeSerializer<N> namespaceSerializer,
 			ListStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
@@ -830,7 +832,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	protected <N, T> ReducingState<T> createReducingState(TypeSerializer<N> namespaceSerializer,
+	protected <N, T> InternalReducingState<N, T> createReducingState(
+			TypeSerializer<N> namespaceSerializer,
 			ReducingStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
@@ -839,7 +842,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	protected <N, T, ACC> FoldingState<T, ACC> createFoldingState(TypeSerializer<N> namespaceSerializer,
+	protected <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
+			TypeSerializer<N> namespaceSerializer,
 			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
index beea81a..e6988f7 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
@@ -23,6 +23,8 @@ import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.internal.InternalListState;
+
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
@@ -30,6 +32,7 @@ import org.rocksdb.WriteOptions;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 /**
@@ -45,7 +48,7 @@ import java.util.List;
  */
 public class RocksDBListState<K, N, V>
 	extends AbstractRocksDBState<K, N, ListState<V>, ListStateDescriptor<V>, V>
-	implements ListState<V> {
+	implements InternalListState<N, V> {
 
 	/** Serializer for the values */
 	private final TypeSerializer<V> valueSerializer;
@@ -117,4 +120,41 @@ public class RocksDBListState<K, N, V>
 		}
 	}
 
+	@Override
+	public void mergeNamespaces(N target, Collection<N> sources) throws Exception {
+		if (sources == null || sources.isEmpty()) {
+			return;
+		}
+
+		// cache key and namespace
+		final K key = backend.getCurrentKey();
+		final int keyGroup = backend.getCurrentKeyGroupIndex();
+
+		try {
+			// create the target full-binary-key 
+			writeKeyWithGroupAndNamespace(
+					keyGroup, key, target,
+					keySerializationStream, keySerializationDataOutputView);
+			final byte[] targetKey = keySerializationStream.toByteArray();
+
+			// merge the sources to the target
+			for (N source : sources) {
+				if (source != null) {
+					writeKeyWithGroupAndNamespace(
+							keyGroup, key, source,
+							keySerializationStream, keySerializationDataOutputView);
+
+					byte[] sourceKey = keySerializationStream.toByteArray();
+					byte[] valueBytes = backend.db.get(columnFamily, sourceKey);
+
+					if (valueBytes != null) {
+						backend.db.merge(columnFamily, writeOptions, targetKey, valueBytes);
+					}
+				}
+			}
+		}
+		catch (Exception e) {
+			throw new Exception("Error while merging state in RocksDB", e);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
index 068c051..ccc98a7 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBReducingState.java
@@ -22,14 +22,18 @@ import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.util.Collection;
 
 /**
  * {@link ReducingState} implementation that stores state in RocksDB.
@@ -40,7 +44,7 @@ import java.io.IOException;
  */
 public class RocksDBReducingState<K, N, V>
 	extends AbstractRocksDBState<K, N, ReducingState<V>, ReducingStateDescriptor<V>, V>
-	implements ReducingState<V> {
+	implements InternalReducingState<N, V> {
 
 	/** Serializer for the values */
 	private final TypeSerializer<V> valueSerializer;
@@ -113,4 +117,72 @@ public class RocksDBReducingState<K, N, V>
 		}
 	}
 
+	@Override
+	public void mergeNamespaces(N target, Collection<N> sources) throws Exception {
+		if (sources == null || sources.isEmpty()) {
+			return;
+		}
+
+		// cache key and namespace
+		final K key = backend.getCurrentKey();
+		final int keyGroup = backend.getCurrentKeyGroupIndex();
+
+		try {
+			V current = null;
+
+			// merge the sources to the target
+			for (N source : sources) {
+				if (source != null) {
+
+					writeKeyWithGroupAndNamespace(
+							keyGroup, key, source,
+							keySerializationStream, keySerializationDataOutputView);
+
+					final byte[] sourceKey = keySerializationStream.toByteArray();
+					final byte[] valueBytes = backend.db.get(columnFamily, sourceKey);
+
+					if (valueBytes != null) {
+						V value = valueSerializer.deserialize(
+								new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes)));
+
+						if (current != null) {
+							current = reduceFunction.reduce(current, value);
+						}
+						else {
+							current = value;
+						}
+					}
+				}
+			}
+
+			// if something came out of merging the sources, merge it or write it to the target
+			if (current != null) {
+				// create the target full-binary-key 
+				writeKeyWithGroupAndNamespace(
+						keyGroup, key, target,
+						keySerializationStream, keySerializationDataOutputView);
+
+				final byte[] targetKey = keySerializationStream.toByteArray();
+				final byte[] targetValueBytes = backend.db.get(columnFamily, targetKey);
+
+				if (targetValueBytes != null) {
+					// target also had a value, merge
+					V value = valueSerializer.deserialize(
+							new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(targetValueBytes)));
+
+					current = reduceFunction.reduce(current, value);
+				}
+
+				// serialize the resulting value
+				keySerializationStream.reset();
+				valueSerializer.serialize(current, keySerializationDataOutputView);
+
+				// write the resulting value
+				backend.db.put(columnFamily, writeOptions, targetKey, keySerializationStream.toByteArray());
+			}
+		}
+		catch (Exception e) {
+			throw new Exception("Error while merging state in RocksDB", e);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
index 9563ed8..7724f02 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
@@ -40,7 +41,7 @@ import java.io.IOException;
  */
 public class RocksDBValueState<K, N, V>
 	extends AbstractRocksDBState<K, N, ValueState<V>, ValueStateDescriptor<V>, V>
-	implements ValueState<V> {
+	implements InternalValueState<N, V> {
 
 	/** Serializer for the values */
 	private final TypeSerializer<V> valueSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBListStateTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBListStateTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBListStateTest.java
new file mode 100644
index 0000000..d8d0308
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBListStateTest.java
@@ -0,0 +1,232 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the {@link ListState} implementation on top of RocksDB.
+ */
+public class RocksDBListStateTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final ListStateDescriptor<Long> stateDescr = new ListStateDescriptor<>("my-state", Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+		
+		try {
+			InternalListState<VoidNamespace, Long> state = 
+					keyedBackend.createListState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(asList(17L, 11L), state.get());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(asList(17L, 11L), state.get());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(asList(1L, 2L, 3L, 2L, 1L), state.get());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final ListStateDescriptor<Long> stateDescr = new ListStateDescriptor<>("my-state", Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final TimeWindow win1 = new TimeWindow(1000, 2000);
+		final TimeWindow win2 = new TimeWindow(2000, 3000);
+		final TimeWindow win3 = new TimeWindow(3000, 4000);
+
+		final Set<Long> expectedResult = new HashSet<>(asList(11L, 22L, 33L, 44L, 55L));
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+
+		try {
+			InternalListState<TimeWindow, Long> state = keyedBackend.createListState(new TimeWindow.Serializer(), stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(win1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(win2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(win3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(win3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(win3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			validateResult(state.get(), expectedResult);
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			validateResult(state.get(), expectedResult);
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static RocksDBKeyedStateBackend<String> createKeyedBackend(RocksDBStateBackend backend) throws Exception {
+		return (RocksDBKeyedStateBackend<String>) backend.createKeyedStateBackend(
+				new DummyEnvironment("TestTask", 1, 0),
+				new JobID(),
+				"test-op",
+				StringSerializer.INSTANCE,
+				16,
+				new KeyGroupRange(2, 3),
+				mock(TaskKvStateRegistry.class));
+	}
+
+	private static <T> void validateResult(Iterable<T> values, Set<T> expected) {
+		int num = 0;
+		for (T v : values) {
+			num++;
+			assertTrue(expected.contains(v));
+		}
+
+		assertEquals(expected.size(), num);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBReducingStateTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBReducingStateTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBReducingStateTest.java
new file mode 100644
index 0000000..fb854f2
--- /dev/null
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBReducingStateTest.java
@@ -0,0 +1,235 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for the {@link ReducingState} implementation on top of RocksDB.
+ */
+public class RocksDBReducingStateTest {
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testAddAndGet() throws Exception {
+
+		final ReducingStateDescriptor<Long> stateDescr = 
+				new ReducingStateDescriptor<>("my-state", new AddingFunction(), Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+		
+		try {
+			InternalReducingState<VoidNamespace, Long> state = 
+					keyedBackend.createReducingState(VoidNamespaceSerializer.INSTANCE, stateDescr);
+			state.setCurrentNamespace(VoidNamespace.INSTANCE);
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+			state.add(17L);
+			state.add(11L);
+			assertEquals(28L, state.get().longValue());
+
+			keyedBackend.setCurrentKey("abc");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertNull(state.get());
+			state.add(1L);
+			state.add(2L);
+
+			keyedBackend.setCurrentKey("def");
+			assertEquals(28L, state.get().longValue());
+			state.clear();
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			state.add(3L);
+			state.add(2L);
+			state.add(1L);
+
+			keyedBackend.setCurrentKey("def");
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("g");
+			assertEquals(9L, state.get().longValue());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	@Test
+	public void testMerging() throws Exception {
+
+		final ReducingStateDescriptor<Long> stateDescr = new ReducingStateDescriptor<>(
+				"my-state", new AddingFunction(), Long.class);
+		stateDescr.initializeSerializerUnlessSet(new ExecutionConfig());
+
+		final TimeWindow win1 = new TimeWindow(1000, 2000);
+		final TimeWindow win2 = new TimeWindow(2000, 3000);
+		final TimeWindow win3 = new TimeWindow(3000, 4000);
+
+		final Long expectedResult = 165L;
+
+		final RocksDBStateBackend backend = new RocksDBStateBackend(tmp.newFolder().toURI());
+		backend.setDbStoragePath(tmp.newFolder().getAbsolutePath());
+
+		final RocksDBKeyedStateBackend<String> keyedBackend = createKeyedBackend(backend);
+
+		try {
+			final InternalReducingState<TimeWindow, Long> state = 
+					keyedBackend.createReducingState(new TimeWindow.Serializer(), stateDescr);
+
+			// populate the different namespaces
+			//  - abc spreads the values over three namespaces
+			//  - def spreads teh values over two namespaces (one empty)
+			//  - ghi is empty
+			//  - jkl has all elements already in the target namespace
+			//  - mno has all elements already in one source namespace
+
+			keyedBackend.setCurrentKey("abc");
+			state.setCurrentNamespace(win1);
+			state.add(33L);
+			state.add(55L);
+
+			state.setCurrentNamespace(win2);
+			state.add(22L);
+			state.add(11L);
+
+			state.setCurrentNamespace(win3);
+			state.add(44L);
+
+			keyedBackend.setCurrentKey("def");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(44L);
+
+			state.setCurrentNamespace(win3);
+			state.add(22L);
+			state.add(55L);
+			state.add(33L);
+
+			keyedBackend.setCurrentKey("jkl");
+			state.setCurrentNamespace(win1);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("mno");
+			state.setCurrentNamespace(win3);
+			state.add(11L);
+			state.add(22L);
+			state.add(33L);
+			state.add(44L);
+			state.add(55L);
+
+			keyedBackend.setCurrentKey("abc");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("def");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("ghi");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertNull(state.get());
+
+			keyedBackend.setCurrentKey("jkl");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+
+			keyedBackend.setCurrentKey("mno");
+			state.mergeNamespaces(win1, asList(win2, win3));
+			state.setCurrentNamespace(win1);
+			assertEquals(expectedResult, state.get());
+		}
+		finally {
+			keyedBackend.close();
+			keyedBackend.dispose();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	private static RocksDBKeyedStateBackend<String> createKeyedBackend(RocksDBStateBackend backend) throws Exception {
+		return (RocksDBKeyedStateBackend<String>) backend.createKeyedStateBackend(
+				new DummyEnvironment("TestTask", 1, 0),
+				new JobID(),
+				"test-op",
+				StringSerializer.INSTANCE,
+				16,
+				new KeyGroupRange(2, 3),
+				mock(TaskKvStateRegistry.class));
+	}
+
+	// ------------------------------------------------------------------------
+	//  test functions
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static class AddingFunction implements ReduceFunction<Long> {
+
+		@Override
+		public Long reduce(Long a, Long b)  {
+			return a + b;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index 8874eca..1675365 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -36,7 +36,7 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 
 import java.util.Map;
@@ -151,7 +151,7 @@ public interface Environment {
 	AccumulatorRegistry getAccumulatorRegistry();
 
 	/**
-	 * Returns the registry for {@link KvState} instances.
+	 * Returns the registry for {@link InternalKvState} instances.
 	 *
 	 * @return KvState registry
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index 3aba6dc..5cf2c26 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.util.Preconditions;
@@ -60,10 +61,10 @@ public class NetworkEnvironment {
 
 	private final TaskEventDispatcher taskEventDispatcher;
 
-	/** Server for {@link org.apache.flink.runtime.state.KvState} requests. */
+	/** Server for {@link InternalKvState} requests. */
 	private final KvStateServer kvStateServer;
 
-	/** Registry for {@link org.apache.flink.runtime.state.KvState} instances. */
+	/** Registry for {@link InternalKvState} instances. */
 	private final KvStateRegistry kvStateRegistry;
 
 	private final IOManager.IOMode defaultIOMode;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index 5ab68fe..de7646b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -38,8 +38,8 @@ import org.apache.flink.runtime.query.KvStateLocation;
 import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.registration.RegistrationResponse;
 import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
@@ -140,10 +140,10 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
 		final Exception cause);
 
 	/**
-	 * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
+	 * Requests a {@link KvStateLocation} for the specified {@link InternalKvState} registration name.
 	 *
 	 * @param registrationName Name under which the KvState has been registered.
-	 * @return Future of the requested {@link KvState} location
+	 * @return Future of the requested {@link InternalKvState} location
 	 */
 	Future<KvStateLocation> lookupKvStateLocation(final String registrationName);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
index bb05833..c122508 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateID.java
@@ -18,11 +18,11 @@
 
 package org.apache.flink.runtime.query;
 
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.AbstractID;
 
 /**
- * Identifier for {@link KvState} instances.
+ * Identifier for {@link InternalKvState} instances.
  *
  * <p>Assigned when registering state at the {@link KvStateRegistry}.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocation.java
index 90bb2a5..f074e8a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocation.java
@@ -21,14 +21,14 @@ package org.apache.flink.runtime.query;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
 import java.util.Arrays;
 
 /**
- * Location information for all key groups of a {@link KvState} instance.
+ * Location information for all key groups of a {@link InternalKvState} instance.
  *
  * <p>This is populated by the {@link KvStateLocationRegistry} and used by the
  * {@link QueryableStateClient} to target queries.

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
index c489025..cb61905 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationRegistry.java
@@ -23,14 +23,14 @@ import org.apache.flink.runtime.execution.SuppressRestartsException;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
 import java.util.HashMap;
 import java.util.Map;
 
 /**
- * Simple registry, which maps {@link KvState} registration notifications to
+ * Simple registry, which maps {@link InternalKvState} registration notifications to
  * {@link KvStateLocation} instances.
  */
 public class KvStateLocationRegistry {

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
index 9ac2d44..6808c5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateMessage.java
@@ -20,14 +20,14 @@ package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
 
 /**
- * Actor messages for {@link KvState} lookup and registration.
+ * Actor messages for {@link InternalKvState} lookup and registration.
  */
 public interface KvStateMessage extends Serializable {
 
@@ -47,7 +47,7 @@ public interface KvStateMessage extends Serializable {
 
 		/**
 		 * Requests a {@link KvStateLocation} for the specified JobID and
-		 * {@link KvState} registration name.
+		 * {@link InternalKvState} registration name.
 		 *
 		 * @param jobId            JobID the KvState instance belongs to
 		 * @param registrationName Name under which the KvState has been registered
@@ -111,7 +111,7 @@ public interface KvStateMessage extends Serializable {
 		private final KvStateServerAddress kvStateServerAddress;
 
 		/**
-		 * Notifies the JobManager about a registered {@link KvState} instance.
+		 * Notifies the JobManager about a registered {@link InternalKvState} instance.
 		 *
 		 * @param jobId                JobID the KvState instance belongs to
 		 * @param jobVertexId          JobVertexID the KvState instance belongs to
@@ -221,7 +221,7 @@ public interface KvStateMessage extends Serializable {
 		private final String registrationName;
 
 		/**
-		 * Notifies the JobManager about an unregistered {@link KvState} instance.
+		 * Notifies the JobManager about an unregistered {@link InternalKvState} instance.
 		 *
 		 * @param jobId                JobID the KvState instance belongs to
 		 * @param jobVertexId          JobVertexID the KvState instance belongs to

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java
index f57ae47..aa698e3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java
@@ -21,15 +21,15 @@ package org.apache.flink.runtime.query;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.runtime.taskmanager.Task;
 
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * A registry for {@link KvState} instances per task manager.
+ * A registry for {@link InternalKvState} instances per task manager.
  *
  * <p>This is currently only used for KvState queries: KvState instances, which
  * are marked as queryable in their state descriptor are registered here and
@@ -41,7 +41,7 @@ import java.util.concurrent.atomic.AtomicReference;
 public class KvStateRegistry {
 
 	/** All registered KvState instances. */
-	private final ConcurrentHashMap<KvStateID, KvState<?>> registeredKvStates =
+	private final ConcurrentHashMap<KvStateID, InternalKvState<?>> registeredKvStates =
 			new ConcurrentHashMap<>();
 
 	/** Registry listener to be notified on registration/unregistration. */
@@ -91,7 +91,7 @@ public class KvStateRegistry {
 			JobVertexID jobVertexId,
 			KeyGroupRange keyGroupRange,
 			String registrationName,
-			KvState<?> kvState) {
+			InternalKvState<?> kvState) {
 
 		KvStateID kvStateId = new KvStateID();
 
@@ -145,7 +145,7 @@ public class KvStateRegistry {
 	 * @param kvStateId KvStateID to identify the KvState instance
 	 * @return KvState instance identified by the KvStateID or <code>null</code>
 	 */
-	public KvState<?> getKvState(KvStateID kvStateId) {
+	public InternalKvState<?> getKvState(KvStateID kvStateId) {
 		return registeredKvStates.get(kvStateId);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
index d831214..e3cf151 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/TaskKvStateRegistry.java
@@ -20,8 +20,8 @@ package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.util.Preconditions;
 
 import java.util.ArrayList;
@@ -59,7 +59,7 @@ public class TaskKvStateRegistry {
 	 *                         descriptor used to create the KvState instance)
 	 * @param kvState          The
 	 */
-	public void registerKvState(KeyGroupRange keyGroupRange, String registrationName, KvState<?> kvState) {
+	public void registerKvState(KeyGroupRange keyGroupRange, String registrationName, InternalKvState<?> kvState) {
 		KvStateID kvStateId = registry.registerKvState(jobId, jobVertexId, keyGroupRange, registrationName, kvState);
 		registeredKvStates.add(new KvStateInfo(keyGroupRange, registrationName, kvStateId));
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
index 34cf15f..92e7658 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.netty.message.KvStateRequest;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,7 +40,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 /**
- * This handler dispatches asynchronous tasks, which query {@link KvState}
+ * This handler dispatches asynchronous tasks, which query {@link InternalKvState}
  * instances and write the result to the channel.
  *
  * <p>The network threads receive the message, deserialize it and dispatch the
@@ -104,7 +104,7 @@ class KvStateServerHandler extends ChannelInboundHandlerAdapter {
 
 				stats.reportRequest();
 
-				KvState<?> kvState = registry.getKvState(request.getKvStateId());
+				InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
 
 				if (kvState != null) {
 					// Execute actual query async, because it is possibly
@@ -179,7 +179,7 @@ class KvStateServerHandler extends ChannelInboundHandlerAdapter {
 	}
 
 	/**
-	 * Task to execute the actual query against the {@link KvState} instance.
+	 * Task to execute the actual query against the {@link InternalKvState} instance.
 	 */
 	private static class AsyncKvStateQueryTask implements Runnable {
 
@@ -187,7 +187,7 @@ class KvStateServerHandler extends ChannelInboundHandlerAdapter {
 
 		private final KvStateRequest request;
 
-		private final KvState<?> kvState;
+		private final InternalKvState<?> kvState;
 
 		private final KvStateRequestStats stats;
 
@@ -196,7 +196,7 @@ class KvStateServerHandler extends ChannelInboundHandlerAdapter {
 		public AsyncKvStateQueryTask(
 				ChannelHandlerContext ctx,
 				KvStateRequest request,
-				KvState<?> kvState,
+				InternalKvState<?> kvState,
 				KvStateRequestStats stats) {
 
 			this.ctx = Objects.requireNonNull(ctx, "Channel handler context");

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java
index 0abb653..4b73fbb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java
@@ -19,11 +19,11 @@
 package org.apache.flink.runtime.query.netty.message;
 
 import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.state.KvState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
 /**
- * A {@link KvState} instance request for a specific key and namespace.
+ * A {@link InternalKvState} instance request for a specific key and namespace.
  */
 public final class KvStateRequest {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/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 cab2b4f..fcca77a 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
@@ -19,12 +19,10 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
-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;
@@ -35,14 +33,18 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.internal.InternalFoldingState;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * Base implementation of KeyedStateBackend. The state can be checkpointed
@@ -63,13 +65,13 @@ public abstract class AbstractKeyedStateBackend<K>
 	private int currentKeyGroup;
 
 	/** So that we can give out state when the user uses the same key. */
-	protected HashMap<String, KvState<?>> keyValueStatesByName;
+	protected final HashMap<String, InternalKvState<?>> keyValueStatesByName;
 
 	/** For caching the last accessed partitioned state */
 	private String lastName;
 
 	@SuppressWarnings("rawtypes")
-	private KvState lastState;
+	private InternalKvState lastState;
 
 	/** The number of key-groups aka max parallelism */
 	protected final int numberOfKeyGroups;
@@ -98,6 +100,7 @@ public abstract class AbstractKeyedStateBackend<K>
 		this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader);
 		this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
 		this.cancelStreamRegistry = new CloseableRegistry();
+		this.keyValueStatesByName = new HashMap<>();
 	}
 
 	/**
@@ -113,7 +116,7 @@ public abstract class AbstractKeyedStateBackend<K>
 
 		lastName = null;
 		lastState = null;
-		keyValueStatesByName = null;
+		keyValueStatesByName.clear();
 	}
 
 	/**
@@ -125,7 +128,9 @@ public abstract class AbstractKeyedStateBackend<K>
 	 * @param <N> The type of the namespace.
 	 * @param <T> The type of the value that the {@code ValueState} can store.
 	 */
-	protected abstract <N, T> ValueState<T> createValueState(TypeSerializer<N> namespaceSerializer, ValueStateDescriptor<T> stateDesc) throws Exception;
+	protected abstract <N, T> InternalValueState<N, T> createValueState(
+			TypeSerializer<N> namespaceSerializer,
+			ValueStateDescriptor<T> stateDesc) throws Exception;
 
 	/**
 	 * Creates and returns a new {@link ListState}.
@@ -136,7 +141,9 @@ public abstract class AbstractKeyedStateBackend<K>
 	 * @param <N> The type of the namespace.
 	 * @param <T> The type of the values that the {@code ListState} can store.
 	 */
-	protected abstract <N, T> ListState<T> createListState(TypeSerializer<N> namespaceSerializer, ListStateDescriptor<T> stateDesc) throws Exception;
+	protected abstract <N, T> InternalListState<N, T> createListState(
+			TypeSerializer<N> namespaceSerializer,
+			ListStateDescriptor<T> stateDesc) throws Exception;
 
 	/**
 	 * Creates and returns a new {@link ReducingState}.
@@ -147,7 +154,9 @@ public abstract class AbstractKeyedStateBackend<K>
 	 * @param <N> The type of the namespace.
 	 * @param <T> The type of the values that the {@code ListState} can store.
 	 */
-	protected abstract <N, T> ReducingState<T> createReducingState(TypeSerializer<N> namespaceSerializer, ReducingStateDescriptor<T> stateDesc) throws Exception;
+	protected abstract <N, T> InternalReducingState<N, T> createReducingState(
+			TypeSerializer<N> namespaceSerializer,
+			ReducingStateDescriptor<T> stateDesc) throws Exception;
 
 	/**
 	 * Creates and returns a new {@link FoldingState}.
@@ -159,7 +168,9 @@ public abstract class AbstractKeyedStateBackend<K>
 	 * @param <T> Type of the values folded into the state
 	 * @param <ACC> Type of the value in the state	 *
 	 */
-	protected abstract <N, T, ACC> FoldingState<T, ACC> createFoldingState(TypeSerializer<N> namespaceSerializer, FoldingStateDescriptor<T, ACC> stateDesc) throws Exception;
+	protected abstract <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
+			TypeSerializer<N> namespaceSerializer,
+			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception;
 
 	/**
 	 * @see KeyedStateBackend
@@ -213,35 +224,27 @@ public abstract class AbstractKeyedStateBackend<K>
 	 * @see KeyedStateBackend
 	 */
 	@Override
-	@SuppressWarnings({"rawtypes", "unchecked"})
-	public <N, S extends State> S getPartitionedState(final N namespace, final TypeSerializer<N> namespaceSerializer, final StateDescriptor<S, ?> stateDescriptor) throws Exception {
-		Preconditions.checkNotNull(namespace, "Namespace");
-		Preconditions.checkNotNull(namespaceSerializer, "Namespace serializer");
+	public <N, S extends State, V> S getOrCreateKeyedState(
+			final TypeSerializer<N> namespaceSerializer,
+			StateDescriptor<S, V> stateDescriptor) throws Exception {
+
+		checkNotNull(namespaceSerializer, "Namespace serializer");
 
 		if (keySerializer == null) {
-			throw new RuntimeException("State key serializer has not been configured in the config. " +
+			throw new UnsupportedOperationException(
+					"State key serializer has not been configured in the config. " +
 					"This operation cannot use partitioned state.");
 		}
-		
-		if (!stateDescriptor.isSerializerInitialized()) {
-			stateDescriptor.initializeSerializerUnlessSet(new ExecutionConfig());
-		}
 
-		if (keyValueStatesByName == null) {
-			keyValueStatesByName = new HashMap<>();
+		if (!stateDescriptor.isSerializerInitialized()) {
+			throw new IllegalStateException("The serializer of the descriptor has not been initialized!"); 
 		}
 
-		if (lastName != null && lastName.equals(stateDescriptor.getName())) {
-			lastState.setCurrentNamespace(namespace);
-			return (S) lastState;
-		}
-
-		KvState<?> previous = keyValueStatesByName.get(stateDescriptor.getName());
-		if (previous != null) {
-			lastState = previous;
-			lastState.setCurrentNamespace(namespace);
-			lastName = stateDescriptor.getName();
-			return (S) previous;
+		InternalKvState<?> existing = keyValueStatesByName.get(stateDescriptor.getName());
+		if (existing != null) {
+			@SuppressWarnings("unchecked")
+			S typedState = (S) existing;
+			return typedState;
 		}
 
 		// create a new blank key/value state
@@ -268,15 +271,10 @@ public abstract class AbstractKeyedStateBackend<K>
 
 		});
 
-		KvState kvState = (KvState) state;
-
+		@SuppressWarnings("unchecked")
+		InternalKvState<N> kvState = (InternalKvState<N>) state;
 		keyValueStatesByName.put(stateDescriptor.getName(), kvState);
 
-		lastName = stateDescriptor.getName();
-		lastState = kvState;
-
-		kvState.setCurrentNamespace(namespace);
-
 		// Publish queryable state
 		if (stateDescriptor.isQueryable()) {
 			if (kvStateRegistry == null) {
@@ -290,90 +288,48 @@ public abstract class AbstractKeyedStateBackend<K>
 		return state;
 	}
 
+	/**
+	 * TODO: NOTE: This method does a lot of work caching / retrieving states just to update the namespace.
+	 *       This method should be removed for the sake of namespaces being lazily fetched from the keyed
+	 *       state backend, or being set on the state directly.
+	 * 
+	 * @see KeyedStateBackend
+	 */
+	@SuppressWarnings("unchecked")
 	@Override
-	public <N, S extends MergingState<?, ?>> void mergePartitionedStates(
-			final N target,
-			Collection<N> sources,
+	public <N, S extends State> S getPartitionedState(
+			final N namespace,
 			final TypeSerializer<N> namespaceSerializer,
 			final StateDescriptor<S, ?> stateDescriptor) throws Exception {
 
-		if (stateDescriptor instanceof ReducingStateDescriptor) {
-			mergeReducingState((ReducingStateDescriptor<?>) stateDescriptor, namespaceSerializer, target,sources);
-		}
-		else if (stateDescriptor instanceof ListStateDescriptor) {
-			mergeListState((ListStateDescriptor<?>) stateDescriptor, namespaceSerializer, target,sources);
-		}
-		else {
-			throw new IllegalArgumentException("Cannot merge states for " + stateDescriptor);
-		}
-	}
+		checkNotNull(namespace, "Namespace");
 
-	private <N, T> void mergeReducingState(
-			final ReducingStateDescriptor<?> stateDescriptor,
-			final TypeSerializer<N> namespaceSerializer,
-			final N target,
-			final Collection<N> sources) throws Exception {
-
-		@SuppressWarnings("unchecked")
-		final ReducingStateDescriptor<T> reducingStateDescriptor = (ReducingStateDescriptor<T>) stateDescriptor;
-
-		@SuppressWarnings("unchecked")
-		final ReducingState<T> state = (ReducingState<T>) getPartitionedState(target, namespaceSerializer, stateDescriptor);
-
-		@SuppressWarnings("unchecked")
-		final KvState<N> kvState = (KvState<N>) state;
-
-		final ReduceFunction<T> reduceFn = reducingStateDescriptor.getReduceFunction();
-
-		T result = null;
-		for (N source: sources) {
-			kvState.setCurrentNamespace(source);
-			T sourceValue = state.get();
-			if (result == null) {
-				result = state.get();
-			} else if (sourceValue != null) {
-				result = reduceFn.reduce(result, sourceValue);
-			}
-			state.clear();
+		// TODO: This is wrong, it should throw an exception that the initialization has not properly happened
+		if (!stateDescriptor.isSerializerInitialized()) {
+			stateDescriptor.initializeSerializerUnlessSet(new ExecutionConfig());
 		}
 
-		// write result to the target
-		kvState.setCurrentNamespace(target);
-		if (result != null) {
-			state.add(result);
+		if (lastName != null && lastName.equals(stateDescriptor.getName())) {
+			lastState.setCurrentNamespace(namespace);
+			return (S) lastState;
 		}
-	}
 
-	private <N, T> void mergeListState(
-			final ListStateDescriptor<?> listStateDescriptor,
-			final TypeSerializer<N> namespaceSerializer,
-			final N target,
-			final Collection<N> sources) throws Exception {
+		InternalKvState<?> previous = keyValueStatesByName.get(stateDescriptor.getName());
+		if (previous != null) {
+			lastState = previous;
+			lastState.setCurrentNamespace(namespace);
+			lastName = stateDescriptor.getName();
+			return (S) previous;
+		}
 
-		@SuppressWarnings("unchecked")
-		final ListState<T> state = (ListState<T>) getPartitionedState(target, namespaceSerializer, listStateDescriptor);
+		final S state = getOrCreateKeyedState(namespaceSerializer, stateDescriptor);
+		final InternalKvState<N> kvState = (InternalKvState<N>) state;
 
-		@SuppressWarnings("unchecked")
-		final KvState<N> kvState = (KvState<N>) state;
-
-		// merge the sources
-		final List<T> result = new ArrayList<>();
-		for (N source: sources) {
-			kvState.setCurrentNamespace(source);
-			Iterable<T> sourceValue = state.get();
-			if (sourceValue != null) {
-				for (T o : sourceValue) {
-					result.add(o);
-				}
-			}
-			state.clear();
-		}
+		lastName = stateDescriptor.getName();
+		lastState = kvState;
+		kvState.setCurrentNamespace(namespace);
 
-		// write to the target
-		kvState.setCurrentNamespace(target);
-		for (T o : result) {
-			state.add(o);
-		}
+		return state;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java
index 03f584e..15e0491 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java
@@ -18,13 +18,10 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
-import java.util.Collection;
-
 /**
  * A keyed state backend provides methods for managing keyed state.
  *
@@ -64,10 +61,31 @@ public interface KeyedStateBackend<K> {
 	TypeSerializer<K> getKeySerializer();
 
 	/**
-	 * Creates or retrieves a partitioned state backed by this state backend.
+	 * Creates or retrieves a keyed state backed by this state backend.
 	 *
+	 * @param namespaceSerializer The serializer used for the namespace type of the state
 	 * @param stateDescriptor The identifier for the state. This contains name and can create a default state value.
+	 *    
+	 * @param <N> The type of the namespace.
+	 * @param <S> The type of the state.
+	 *
+	 * @return A new key/value state backed by this backend.
+	 *
+	 * @throws Exception Exceptions may occur during initialization of the state and should be forwarded.
+	 */
+	<N, S extends State, T> S getOrCreateKeyedState(
+			TypeSerializer<N> namespaceSerializer,
+			StateDescriptor<S, T> stateDescriptor) throws Exception;
 
+	/**
+	 * Creates or retrieves a partitioned state backed by this state backend.
+	 * 
+	 * TODO: NOTE: This method does a lot of work caching / retrieving states just to update the namespace.
+	 *       This method should be removed for the sake of namespaces being lazily fetched from the keyed
+	 *       state backend, or being set on the state directly.
+	 *
+	 * @param stateDescriptor The identifier for the state. This contains name and can create a default state value.
+	 *
 	 * @param <N> The type of the namespace.
 	 * @param <S> The type of the state.
 	 *
@@ -75,20 +93,11 @@ public interface KeyedStateBackend<K> {
 	 *
 	 * @throws Exception Exceptions may occur during initialization of the state and should be forwarded.
 	 */
-	@SuppressWarnings({"rawtypes", "unchecked"})
 	<N, S extends State> S getPartitionedState(
 			N namespace,
 			TypeSerializer<N> namespaceSerializer,
 			StateDescriptor<S, ?> stateDescriptor) throws Exception;
 
-
-	@SuppressWarnings("unchecked,rawtypes")
-	<N, S extends MergingState<?, ?>> void mergePartitionedStates(
-			N target,
-			Collection<N> sources,
-			TypeSerializer<N> namespaceSerializer,
-			StateDescriptor<S, ?> stateDescriptor) throws Exception;
-
 	/**
 	 * Closes the backend and releases all resources.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.java
deleted file mode 100644
index aded79f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KvState.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.state;
-
-/**
- * Key/Value state implementation for user-defined state. The state is backed by a state
- * backend, which typically follows one of the following patterns: Either the state is stored
- * in the key/value state object directly (meaning in the executing JVM) and snapshotted by the
- * state backend into some store (during checkpoints), or the key/value state is in fact backed
- * by an external key/value store as the state backend, and checkpoints merely record the
- * metadata of what is considered part of the checkpoint.
- * 
- * @param <N> The type of the namespace.
- */
-public interface KvState<N> {
-
-	/**
-	 * Sets the current namespace, which will be used when using the state access methods.
-	 *
-	 * @param namespace The namespace.
-	 */
-	void setCurrentNamespace(N namespace);
-
-	/**
-	 * Returns the serialized value for the given key and namespace.
-	 *
-	 * <p>If no value is associated with key and namespace, <code>null</code>
-	 * is returned.
-	 *
-	 * @param serializedKeyAndNamespace Serialized key and namespace
-	 * @return Serialized value or <code>null</code> if no value is associated
-	 * with the key and namespace.
-	 * @throws Exception Exceptions during serialization are forwarded
-	 */
-	byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespace.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespace.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespace.java
index 9ff9df0..fb2e512 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespace.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespace.java
@@ -18,18 +18,53 @@
 
 package org.apache.flink.runtime.state;
 
+import java.io.ObjectStreamException;
+
 /**
- * Uninstantiable placeholder class for state without a namespace.
+ * Singleton placeholder class for state without a namespace.
  */
 public final class VoidNamespace {
 
-	public static final VoidNamespace INSTANCE = new VoidNamespace();
+	// ------------------------------------------------------------------------
+	//  Singleton instance
+	// ------------------------------------------------------------------------
 
-	private VoidNamespace() {
-	}
+	/** The singleton instance */
+	public static final VoidNamespace INSTANCE = new VoidNamespace();
 
+	/** Getter for the singleton instance */
 	public static VoidNamespace get() {
 		return INSTANCE;
 	}
 
+	/** This class should not be instantiated */
+	private VoidNamespace() {}
+
+	// ------------------------------------------------------------------------
+	//  Standard Utilities
+	// ------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return 99;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return obj == this;
+	}
+
+	@Override
+	public String toString() {
+		return getClass().getSimpleName();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Singleton serialization
+	// ------------------------------------------------------------------------
+
+	// make sure that we preserve the singleton properly on serialization
+	private Object readResolve() throws ObjectStreamException {
+		return INSTANCE;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapMergingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapMergingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapMergingState.java
new file mode 100644
index 0000000..4ac7125
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapMergingState.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state.heap;
+
+import org.apache.flink.api.common.state.MergingState;
+import org.apache.flink.api.common.state.State;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalMergingState;
+
+import java.util.Collection;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Base class for {@link MergingState} ({@link org.apache.flink.runtime.state.internal.InternalMergingState})
+ * that is stored on the heap.
+ * 
+ * @param <K> The type of the key.
+ * @param <N> The type of the namespace.
+ * @param <SV> The type of the values in the state.
+ * @param <S> The type of State
+ * @param <SD> The type of StateDescriptor for the State S
+ */
+public abstract class AbstractHeapMergingState<K, N, IN, OUT, SV, S extends State, SD extends StateDescriptor<S, ?>>
+		extends AbstractHeapState<K, N, SV, S, SD>
+		implements InternalMergingState<N, IN, OUT> {
+
+	/**
+	 * Creates a new key/value state for the given hash map of key/value pairs.
+	 *
+	 * @param backend The state backend backing that created this state.
+	 * @param stateDesc The state identifier for the state. This contains name
+	 *                           and can create a default state value.
+	 * @param stateTable The state tab;e to use in this kev/value state. May contain initial state.
+	 */
+	protected AbstractHeapMergingState(
+			KeyedStateBackend<K> backend,
+			SD stateDesc,
+			StateTable<K, N, SV> stateTable,
+			TypeSerializer<K> keySerializer,
+			TypeSerializer<N> namespaceSerializer) {
+
+		super(backend, stateDesc, stateTable, keySerializer, namespaceSerializer);
+	}
+
+	@Override
+	public void mergeNamespaces(N target, Collection<N> sources) throws Exception {
+		if (sources == null || sources.isEmpty()) {
+			return; // nothing to do
+		}
+
+		final K key = backend.getCurrentKey();
+		checkState(key != null, "No key set.");
+
+		final Map<N, Map<K, SV>> namespaceMap = stateTable.get(backend.getCurrentKeyGroupIndex());
+
+		if (namespaceMap != null) {
+			SV merged = null;
+
+			// merge the sources
+			for (N source : sources) {
+				Map<K, SV> keysForNamespace = namespaceMap.get(source);
+				if (keysForNamespace != null) {
+					// get and remove the next source per namespace/key
+					SV sourceState = keysForNamespace.remove(key);
+
+					// if the namespace map became empty, remove 
+					if (keysForNamespace.isEmpty()) {
+						namespaceMap.remove(source);
+					}
+
+					if (merged != null && sourceState != null) {
+						merged = mergeState(merged, sourceState);
+					}
+					else if (merged == null) {
+						merged = sourceState;
+					}
+				}
+			}
+
+			// merge into the target, if needed
+			if (merged != null) {
+				Map<K, SV> keysForTarget = namespaceMap.get(target);
+				if (keysForTarget == null) {
+					keysForTarget = createNewMap();
+					namespaceMap.put(target, keysForTarget);
+				}
+				SV targetState = keysForTarget.get(key);
+
+				if (targetState != null) {
+					targetState = mergeState(targetState, merged);
+				}
+				else {
+					targetState = merged;
+				}
+				keysForTarget.put(key, targetState);
+			}
+		}
+
+		// else no entries for that key at all, nothing to do skip
+	}
+
+	protected abstract SV mergeState(SV a, SV b) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java
index 18d1bc7..18b71de 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java
@@ -24,9 +24,9 @@ import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.KvState;
 import org.apache.flink.util.Preconditions;
 
 import java.util.HashMap;
@@ -44,7 +44,7 @@ import java.util.concurrent.ConcurrentHashMap;
  * @param <SD> The type of StateDescriptor for the State S
  */
 public abstract class AbstractHeapState<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>>
-		implements KvState<N>, State {
+		implements InternalKvState<N> {
 
 	/** Map containing the actual key/value pairs */
 	protected final StateTable<K, N, SV> stateTable;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java
index 1679122..6df3f5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.internal.InternalFoldingState;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
@@ -39,8 +40,9 @@ import java.util.Map;
  */
 public class HeapFoldingState<K, N, T, ACC>
 		extends AbstractHeapState<K, N, ACC, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>>
-		implements FoldingState<T, ACC> {
+		implements InternalFoldingState<N, T, ACC> {
 
+	/** The function used to fold the state */
 	private final FoldFunction<T, ACC> foldFunction;
 
 	/**
@@ -61,6 +63,10 @@ public class HeapFoldingState<K, N, T, ACC>
 		this.foldFunction = stateDesc.getFoldFunction();
 	}
 
+	// ------------------------------------------------------------------------
+	//  state access
+	// ------------------------------------------------------------------------
+
 	@Override
 	public ACC get() {
 		Preconditions.checkState(currentNamespace != null, "No namespace set.");

http://git-wip-us.apache.org/repos/asf/flink/blob/3b97128f/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index 9a9178a..89d4f76 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -19,14 +19,10 @@
 package org.apache.flink.runtime.state.heap;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.flink.api.common.state.FoldingState;
 import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.VoidSerializer;
@@ -54,6 +50,10 @@ import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalFoldingState;
+import org.apache.flink.runtime.state.internal.InternalListState;
+import org.apache.flink.runtime.state.internal.InternalReducingState;
+import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
@@ -133,17 +133,23 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		return stateTable;
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
-	public <N, V> ValueState<V> createValueState(TypeSerializer<N> namespaceSerializer, ValueStateDescriptor<V> stateDesc) throws Exception {
+	public <N, V> InternalValueState<N, V> createValueState(
+			TypeSerializer<N> namespaceSerializer,
+			ValueStateDescriptor<V> stateDesc) throws Exception {
+
 		StateTable<K, N, V> stateTable = tryRegisterStateTable(namespaceSerializer, stateDesc);
 		return new HeapValueState<>(this, stateDesc, stateTable, keySerializer, namespaceSerializer);
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
-	public <N, T> ListState<T> createListState(TypeSerializer<N> namespaceSerializer, ListStateDescriptor<T> stateDesc) throws Exception {
+	public <N, T> InternalListState<N, T> createListState(
+			TypeSerializer<N> namespaceSerializer,
+			ListStateDescriptor<T> stateDesc) throws Exception {
+
 		String name = stateDesc.getName();
+
+		@SuppressWarnings("unchecked")
 		StateTable<K, N, ArrayList<T>> stateTable = (StateTable<K, N, ArrayList<T>>) stateTables.get(name);
 
 		RegisteredBackendStateMetaInfo<N, ArrayList<T>> newMetaInfo =
@@ -153,15 +159,20 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		return new HeapListState<>(this, stateDesc, stateTable, keySerializer, namespaceSerializer);
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
-	public <N, T> ReducingState<T> createReducingState(TypeSerializer<N> namespaceSerializer, ReducingStateDescriptor<T> stateDesc) throws Exception {
+	public <N, T> InternalReducingState<N, T> createReducingState(
+			TypeSerializer<N> namespaceSerializer,
+			ReducingStateDescriptor<T> stateDesc) throws Exception {
+
 		StateTable<K, N, T> stateTable = tryRegisterStateTable(namespaceSerializer, stateDesc);
 		return new HeapReducingState<>(this, stateDesc, stateTable, keySerializer, namespaceSerializer);
 	}
-	@SuppressWarnings("unchecked")
+
 	@Override
-	protected <N, T, ACC> FoldingState<T, ACC> createFoldingState(TypeSerializer<N> namespaceSerializer, FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
+	protected <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
+			TypeSerializer<N> namespaceSerializer,
+			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
+
 		StateTable<K, N, ACC> stateTable = tryRegisterStateTable(namespaceSerializer, stateDesc);
 		return new HeapFoldingState<>(this, stateDesc, stateTable, keySerializer, namespaceSerializer);
 	}


[8/9] flink git commit: [hotfix] Remove sysout logging in SavepointMigrationTestBase and fix several warnings.

Posted by se...@apache.org.
[hotfix] Remove sysout logging in SavepointMigrationTestBase and fix several warnings.


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

Branch: refs/heads/master
Commit: 2b3fd395ef254e8828e173b0afd23a1a9f7033a3
Parents: b8a784e
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jan 20 18:49:57 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Jan 22 21:53:40 2017 +0100

----------------------------------------------------------------------
 .../utils/SavepointMigrationTestBase.java            | 15 ++++++---------
 1 file changed, 6 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2b3fd395/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
index 80a66ac..1a8a0a0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
@@ -103,7 +103,8 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
 	}
 
-	protected void executeAndSavepoint(
+	@SafeVarargs
+	protected final void executeAndSavepoint(
 			StreamExecutionEnvironment env,
 			String savepointPath,
 			Tuple2<String, Integer>... expectedAccumulators) throws Exception {
@@ -188,16 +189,14 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 //		FileUtils.moveFile(new File(new URI(path).getPath()), new File(savepointPath));
 	}
 
-	protected void restoreAndExecute(
+	@SafeVarargs
+	protected final void restoreAndExecute(
 			StreamExecutionEnvironment env,
 			String savepointPath,
 			Tuple2<String, Integer>... expectedAccumulators) throws Exception {
 
-		int parallelism = env.getParallelism();
-
 		// Retrieve the job manager
-
-		ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft());
+		Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft());
 
 		// Submit the job
 		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
@@ -217,17 +216,15 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 			for (Tuple2<String, Integer> acc : expectedAccumulators) {
 				Integer numFinished = (Integer) accumulators.get(acc.f0);
 				if (numFinished == null) {
-					System.out.println("NO ACC FOR " + acc);
 					allDone = false;
 					break;
 				}
 				if (!numFinished.equals(acc.f1)) {
-					System.out.println("TO LOW FOR ACC" + acc);
 					allDone = false;
 					break;
 				}
 			}
-			System.out.println("ACC: " + accumulators);
+
 			if (allDone) {
 				done = true;
 				break;