You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by kk...@apache.org on 2017/10/11 15:46:01 UTC

[01/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Repository: flink
Updated Branches:
  refs/heads/master bc4638a3c -> f48f5340a


http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 2567004..aa4e6d8 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
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 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.VoidNamespace;
@@ -35,10 +34,6 @@ import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
-import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
-
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -53,18 +48,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 
 /**
- * Tests for {@link KvStateRequestSerializer}.
+ * Tests for {@link KvStateSerializer}.
  */
 @RunWith(Parameterized.class)
 public class KvStateRequestSerializerTest {
 
-	private final ByteBufAllocator alloc = UnpooledByteBufAllocator.DEFAULT;
-
 	@Parameterized.Parameters
 	public static Collection<Boolean> parameters() {
 		return Arrays.asList(false, true);
@@ -74,155 +66,6 @@ public class KvStateRequestSerializerTest {
 	public boolean async;
 
 	/**
-	 * Tests KvState request serialization.
-	 */
-	@Test
-	public void testKvStateRequestSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 1337L;
-		KvStateID kvStateId = new KvStateID();
-		byte[] serializedKeyAndNamespace = randomByteArray(1024);
-
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequest(
-				alloc,
-				requestId,
-				kvStateId,
-				serializedKeyAndNamespace);
-
-		int frameLength = buf.readInt();
-		assertEquals(KvStateRequestType.REQUEST, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequest request = KvStateRequestSerializer.deserializeKvStateRequest(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-		assertEquals(kvStateId, request.getKvStateId());
-		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
-	}
-
-	/**
-	 * Tests KvState request serialization with zero-length serialized key and namespace.
-	 */
-	@Test
-	public void testKvStateRequestSerializationWithZeroLengthKeyAndNamespace() throws Exception {
-		byte[] serializedKeyAndNamespace = new byte[0];
-
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequest(
-				alloc,
-				1823,
-				new KvStateID(),
-				serializedKeyAndNamespace);
-
-		int frameLength = buf.readInt();
-		assertEquals(KvStateRequestType.REQUEST, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequest request = KvStateRequestSerializer.deserializeKvStateRequest(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
-	}
-
-	/**
-	 * Tests that we don't try to be smart about <code>null</code> key and namespace.
-	 * They should be treated explicitly.
-	 */
-	@Test(expected = NullPointerException.class)
-	public void testNullPointerExceptionOnNullSerializedKeyAndNamepsace() throws Exception {
-		new KvStateRequest(0, new KvStateID(), null);
-	}
-
-	/**
-	 * Tests KvState request result serialization.
-	 */
-	@Test
-	public void testKvStateRequestResultSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 72727278L;
-		byte[] serializedResult = randomByteArray(1024);
-
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequestResult(
-				alloc,
-				requestId,
-				serializedResult);
-
-		int frameLength = buf.readInt();
-		assertEquals(KvStateRequestType.REQUEST_RESULT, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestResult request = KvStateRequestSerializer.deserializeKvStateRequestResult(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-
-		assertArrayEquals(serializedResult, request.getSerializedResult());
-	}
-
-	/**
-	 * Tests KvState request result serialization with zero-length serialized result.
-	 */
-	@Test
-	public void testKvStateRequestResultSerializationWithZeroLengthSerializedResult() throws Exception {
-		byte[] serializedResult = new byte[0];
-
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequestResult(
-				alloc,
-				72727278,
-				serializedResult);
-
-		int frameLength = buf.readInt();
-
-		assertEquals(KvStateRequestType.REQUEST_RESULT, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestResult request = KvStateRequestSerializer.deserializeKvStateRequestResult(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertArrayEquals(serializedResult, request.getSerializedResult());
-	}
-
-	/**
-	 * Tests that we don't try to be smart about <code>null</code> results.
-	 * They should be treated explicitly.
-	 */
-	@Test(expected = NullPointerException.class)
-	public void testNullPointerExceptionOnNullSerializedResult() throws Exception {
-		new KvStateRequestResult(0, null);
-	}
-
-	/**
-	 * Tests KvState request failure serialization.
-	 */
-	@Test
-	public void testKvStateRequestFailureSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 1111222L;
-		IllegalStateException cause = new IllegalStateException("Expected test");
-
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequestFailure(
-				alloc,
-				requestId,
-				cause);
-
-		int frameLength = buf.readInt();
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure request = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-		assertEquals(cause.getClass(), request.getCause().getClass());
-		assertEquals(cause.getMessage(), request.getCause().getMessage());
-	}
-
-	/**
-	 * Tests KvState server failure serialization.
-	 */
-	@Test
-	public void testServerFailureSerialization() throws Exception {
-		IllegalStateException cause = new IllegalStateException("Expected test");
-
-		ByteBuf buf = KvStateRequestSerializer.serializeServerFailure(alloc, cause);
-
-		int frameLength = buf.readInt();
-		assertEquals(KvStateRequestType.SERVER_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		Throwable request = KvStateRequestSerializer.deserializeServerFailure(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(cause.getClass(), request.getClass());
-		assertEquals(cause.getMessage(), request.getMessage());
-	}
-
-	/**
 	 * Tests key and namespace serialization utils.
 	 */
 	@Test
@@ -233,10 +76,10 @@ public class KvStateRequestSerializerTest {
 		long expectedKey = Integer.MAX_VALUE + 12323L;
 		String expectedNamespace = "knilf";
 
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
 				expectedKey, keySerializer, expectedNamespace, namespaceSerializer);
 
-		Tuple2<Long, String> actual = KvStateRequestSerializer.deserializeKeyAndNamespace(
+		Tuple2<Long, String> actual = KvStateSerializer.deserializeKeyAndNamespace(
 				serializedKeyAndNamespace, keySerializer, namespaceSerializer);
 
 		assertEquals(expectedKey, actual.f0.longValue());
@@ -248,7 +91,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test(expected = IOException.class)
 	public void testKeyAndNamespaceDeserializationEmpty() throws Exception {
-		KvStateRequestSerializer.deserializeKeyAndNamespace(
+		KvStateSerializer.deserializeKeyAndNamespace(
 			new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
 	}
 
@@ -257,7 +100,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test(expected = IOException.class)
 	public void testKeyAndNamespaceDeserializationTooShort() throws Exception {
-		KvStateRequestSerializer.deserializeKeyAndNamespace(
+		KvStateSerializer.deserializeKeyAndNamespace(
 			new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
 	}
 
@@ -267,7 +110,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testKeyAndNamespaceDeserializationTooMany1() throws Exception {
 		// Long + null String + 1 byte
-		KvStateRequestSerializer.deserializeKeyAndNamespace(
+		KvStateSerializer.deserializeKeyAndNamespace(
 			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2}, LongSerializer.INSTANCE,
 			StringSerializer.INSTANCE);
 	}
@@ -278,7 +121,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testKeyAndNamespaceDeserializationTooMany2() throws Exception {
 		// Long + null String + 2 bytes
-		KvStateRequestSerializer.deserializeKeyAndNamespace(
+		KvStateSerializer.deserializeKeyAndNamespace(
 			new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 42, 0, 2, 2}, LongSerializer.INSTANCE,
 			StringSerializer.INSTANCE);
 	}
@@ -291,8 +134,8 @@ public class KvStateRequestSerializerTest {
 		TypeSerializer<Long> valueSerializer = LongSerializer.INSTANCE;
 		long expectedValue = Long.MAX_VALUE - 1292929292L;
 
-		byte[] serializedValue = KvStateRequestSerializer.serializeValue(expectedValue, valueSerializer);
-		long actualValue = KvStateRequestSerializer.deserializeValue(serializedValue, valueSerializer);
+		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
+		long actualValue = KvStateSerializer.deserializeValue(serializedValue, valueSerializer);
 
 		assertEquals(expectedValue, actualValue);
 	}
@@ -302,7 +145,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test(expected = IOException.class)
 	public void testDeserializeValueEmpty() throws Exception {
-		KvStateRequestSerializer.deserializeValue(new byte[] {}, LongSerializer.INSTANCE);
+		KvStateSerializer.deserializeValue(new byte[] {}, LongSerializer.INSTANCE);
 	}
 
 	/**
@@ -311,7 +154,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeValueTooShort() throws Exception {
 		// 1 byte (incomplete Long)
-		KvStateRequestSerializer.deserializeValue(new byte[] {1}, LongSerializer.INSTANCE);
+		KvStateSerializer.deserializeValue(new byte[] {1}, LongSerializer.INSTANCE);
 	}
 
 	/**
@@ -320,7 +163,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeValueTooMany1() throws Exception {
 		// Long + 1 byte
-		KvStateRequestSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2},
+		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2},
 			LongSerializer.INSTANCE);
 	}
 
@@ -330,7 +173,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeValueTooMany2() throws Exception {
 		// Long + 2 bytes
-		KvStateRequestSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 2},
+		KvStateSerializer.deserializeValue(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 2},
 			LongSerializer.INSTANCE);
 	}
 
@@ -363,7 +206,7 @@ public class KvStateRequestSerializerTest {
 
 	/**
 	 * Verifies that the serialization of a list using the given list state
-	 * matches the deserialization with {@link KvStateRequestSerializer#deserializeList}.
+	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
 	 *
 	 * @param key
 	 * 		key of the list state
@@ -390,19 +233,19 @@ public class KvStateRequestSerializerTest {
 		}
 
 		final byte[] serializedKey =
-			KvStateRequestSerializer.serializeKeyAndNamespace(
+			KvStateSerializer.serializeKeyAndNamespace(
 				key, LongSerializer.INSTANCE,
 				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
 
 		final byte[] serializedValues = listState.getSerializedValue(serializedKey);
 
-		List<Long> actualValues = KvStateRequestSerializer.deserializeList(serializedValues, valueSerializer);
+		List<Long> actualValues = KvStateSerializer.deserializeList(serializedValues, valueSerializer);
 		assertEquals(expectedValues, actualValues);
 
 		// Single value
 		long expectedValue = ThreadLocalRandom.current().nextLong();
-		byte[] serializedValue = KvStateRequestSerializer.serializeValue(expectedValue, valueSerializer);
-		List<Long> actualValue = KvStateRequestSerializer.deserializeList(serializedValue, valueSerializer);
+		byte[] serializedValue = KvStateSerializer.serializeValue(expectedValue, valueSerializer);
+		List<Long> actualValue = KvStateSerializer.deserializeList(serializedValue, valueSerializer);
 		assertEquals(1, actualValue.size());
 		assertEquals(expectedValue, actualValue.get(0).longValue());
 	}
@@ -412,7 +255,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test
 	public void testDeserializeListEmpty() throws Exception {
-		List<Long> actualValue = KvStateRequestSerializer
+		List<Long> actualValue = KvStateSerializer
 			.deserializeList(new byte[] {}, LongSerializer.INSTANCE);
 		assertEquals(0, actualValue.size());
 	}
@@ -423,7 +266,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeListTooShort1() throws Exception {
 		// 1 byte (incomplete Long)
-		KvStateRequestSerializer.deserializeList(new byte[] {1}, LongSerializer.INSTANCE);
+		KvStateSerializer.deserializeList(new byte[] {1}, LongSerializer.INSTANCE);
 	}
 
 	/**
@@ -432,7 +275,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeListTooShort2() throws Exception {
 		// Long + 1 byte (separator) + 1 byte (incomplete Long)
-		KvStateRequestSerializer.deserializeList(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 3},
+		KvStateSerializer.deserializeList(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 2, 3},
 			LongSerializer.INSTANCE);
 	}
 
@@ -466,7 +309,7 @@ public class KvStateRequestSerializerTest {
 
 	/**
 	 * Verifies that the serialization of a map using the given map state
-	 * matches the deserialization with {@link KvStateRequestSerializer#deserializeList}.
+	 * matches the deserialization with {@link KvStateSerializer#deserializeList}.
 	 *
 	 * @param key
 	 * 		key of the map state
@@ -497,13 +340,13 @@ public class KvStateRequestSerializerTest {
 		mapState.put(0L, null);
 
 		final byte[] serializedKey =
-			KvStateRequestSerializer.serializeKeyAndNamespace(
+			KvStateSerializer.serializeKeyAndNamespace(
 				key, LongSerializer.INSTANCE,
 				VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE);
 
 		final byte[] serializedValues = mapState.getSerializedValue(serializedKey);
 
-		Map<Long, String> actualValues = KvStateRequestSerializer.deserializeMap(serializedValues, userKeySerializer, userValueSerializer);
+		Map<Long, String> actualValues = KvStateSerializer.deserializeMap(serializedValues, userKeySerializer, userValueSerializer);
 		assertEquals(expectedValues.size(), actualValues.size());
 		for (Map.Entry<Long, String> actualEntry : actualValues.entrySet()) {
 			assertEquals(expectedValues.get(actualEntry.getKey()), actualEntry.getValue());
@@ -515,12 +358,12 @@ public class KvStateRequestSerializerTest {
 		String expectedValue = Long.toString(expectedKey);
 		byte[] isNull = {0};
 
-		baos.write(KvStateRequestSerializer.serializeValue(expectedKey, userKeySerializer));
+		baos.write(KvStateSerializer.serializeValue(expectedKey, userKeySerializer));
 		baos.write(isNull);
-		baos.write(KvStateRequestSerializer.serializeValue(expectedValue, userValueSerializer));
+		baos.write(KvStateSerializer.serializeValue(expectedValue, userValueSerializer));
 		byte[] serializedValue = baos.toByteArray();
 
-		Map<Long, String> actualValue = KvStateRequestSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
+		Map<Long, String> actualValue = KvStateSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
 		assertEquals(1, actualValue.size());
 		assertEquals(expectedValue, actualValue.get(expectedKey));
 	}
@@ -530,7 +373,7 @@ public class KvStateRequestSerializerTest {
 	 */
 	@Test
 	public void testDeserializeMapEmpty() throws Exception {
-		Map<Long, String> actualValue = KvStateRequestSerializer
+		Map<Long, String> actualValue = KvStateSerializer
 			.deserializeMap(new byte[] {}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
 		assertEquals(0, actualValue.size());
 	}
@@ -541,7 +384,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeMapTooShort1() throws Exception {
 		// 1 byte (incomplete Key)
-		KvStateRequestSerializer.deserializeMap(new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
+		KvStateSerializer.deserializeMap(new byte[] {1}, LongSerializer.INSTANCE, StringSerializer.INSTANCE);
 	}
 
 	/**
@@ -550,7 +393,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeMapTooShort2() throws Exception {
 		// Long (Key) + 1 byte (incomplete Value)
-		KvStateRequestSerializer.deserializeMap(new byte[]{1, 1, 1, 1, 1, 1, 1, 1, 0},
+		KvStateSerializer.deserializeMap(new byte[]{1, 1, 1, 1, 1, 1, 1, 1, 0},
 				LongSerializer.INSTANCE, LongSerializer.INSTANCE);
 	}
 
@@ -560,7 +403,7 @@ public class KvStateRequestSerializerTest {
 	@Test(expected = IOException.class)
 	public void testDeserializeMapTooShort3() throws Exception {
 		// Long (Key1) + Boolean (false) + Long (Value1) + 1 byte (incomplete Key2)
-		KvStateRequestSerializer.deserializeMap(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 3},
+		KvStateSerializer.deserializeMap(new byte[] {1, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 3},
 			LongSerializer.INSTANCE, LongSerializer.INSTANCE);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 ed280a7..dbf131f 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
@@ -56,7 +56,7 @@ import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
 import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.KvStateRegistryListener;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.heap.AbstractHeapState;
 import org.apache.flink.runtime.state.heap.NestedMapsStateTable;
 import org.apache.flink.runtime.state.heap.StateTable;
@@ -3070,7 +3070,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			TypeSerializer<N> namespaceSerializer,
 			TypeSerializer<V> valueSerializer) throws Exception {
 
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
 				key, keySerializer, namespace, namespaceSerializer);
 
 		byte[] serializedValue = kvState.getSerializedValue(serializedKeyAndNamespace);
@@ -3078,7 +3078,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		if (serializedValue == null) {
 			return null;
 		} else {
-			return KvStateRequestSerializer.deserializeValue(serializedValue, valueSerializer);
+			return KvStateSerializer.deserializeValue(serializedValue, valueSerializer);
 		}
 	}
 
@@ -3094,7 +3094,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			TypeSerializer<N> namespaceSerializer,
 			TypeSerializer<V> valueSerializer) throws Exception {
 
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
 				key, keySerializer, namespace, namespaceSerializer);
 
 		byte[] serializedValue = kvState.getSerializedValue(serializedKeyAndNamespace);
@@ -3102,7 +3102,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		if (serializedValue == null) {
 			return null;
 		} else {
-			return KvStateRequestSerializer.deserializeList(serializedValue, valueSerializer);
+			return KvStateSerializer.deserializeList(serializedValue, valueSerializer);
 		}
 	}
 
@@ -3120,7 +3120,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			TypeSerializer<UV> userValueSerializer
 	) throws Exception {
 
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
 				key, keySerializer, namespace, namespaceSerializer);
 
 		byte[] serializedValue = kvState.getSerializedValue(serializedKeyAndNamespace);
@@ -3128,7 +3128,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		if (serializedValue == null) {
 			return null;
 		} else {
-			return KvStateRequestSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
+			return KvStateSerializer.deserializeMap(serializedValue, userKeySerializer, userValueSerializer);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
deleted file mode 100644
index 8ac3d2f..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
+++ /dev/null
@@ -1,1128 +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.test.query;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
-import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
-import org.apache.flink.runtime.query.QueryableStateClient;
-import org.apache.flink.runtime.query.netty.UnknownKeyOrNamespace;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceTypeInfo;
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.QueryableStateStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.OnSuccess;
-import akka.dispatch.Recover;
-import akka.pattern.Patterns;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicLongArray;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-import scala.reflect.ClassTag$;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Base class for queryable state integration tests with a configurable state backend.
- */
-public abstract class AbstractQueryableStateITCase extends TestLogger {
-
-	protected static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000, TimeUnit.SECONDS);
-	private static final FiniteDuration QUERY_RETRY_DELAY = new FiniteDuration(100, TimeUnit.MILLISECONDS);
-
-	protected static ActorSystem testActorSystem;
-
-	/**
-	 * State backend to use.
-	 */
-	protected AbstractStateBackend stateBackend;
-
-	/**
-	 * Shared between all the test. Make sure to have at least NUM_SLOTS
-	 * available after your test finishes, e.g. cancel the job you submitted.
-	 */
-	protected static FlinkMiniCluster cluster;
-
-	protected static int maxParallelism;
-
-	@Before
-	public void setUp() throws Exception {
-		// NOTE: do not use a shared instance for all tests as the tests may brake
-		this.stateBackend = createStateBackend();
-
-		Assert.assertNotNull(cluster);
-
-		maxParallelism = cluster.configuration().getInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1) *
-				cluster.configuration().getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-	}
-
-	/**
-	 * Creates a state backend instance which is used in the {@link #setUp()} method before each
-	 * test case.
-	 *
-	 * @return a state backend instance for each unit test
-	 */
-	protected abstract AbstractStateBackend createStateBackend() throws Exception;
-
-	/**
-	 * Runs a simple topology producing random (key, 1) pairs at the sources (where
-	 * number of keys is in fixed in range 0...numKeys). The records are keyed and
-	 * a reducing queryable state instance is created, which sums up the records.
-	 *
-	 * <p>After submitting the job in detached mode, the QueryableStateCLient is used
-	 * to query the counts of each key in rounds until all keys have non-zero counts.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testQueryableState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-		final int numKeys = 256;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-
-		try {
-			//
-			// Test program
-			//
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestKeyRangeSource(numKeys));
-
-			// Reducing state
-			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState = new ReducingStateDescriptor<>(
-					"any-name",
-					new SumReduce(),
-					source.getType());
-
-			final String queryName = "hakuna-matata";
-
-			final QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 7143749578983540352L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState(queryName, reducingState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			cluster.submitJobDetached(jobGraph);
-
-			//
-			// Start querying
-			//
-			jobId = jobGraph.getJobID();
-
-			final AtomicLongArray counts = new AtomicLongArray(numKeys);
-
-			boolean allNonZero = false;
-			while (!allNonZero && deadline.hasTimeLeft()) {
-				allNonZero = true;
-
-				final List<Future<Tuple2<Integer, Long>>> futures = new ArrayList<>(numKeys);
-
-				for (int i = 0; i < numKeys; i++) {
-					final int key = i;
-
-					if (counts.get(key) > 0) {
-						// Skip this one
-						continue;
-					} else {
-						allNonZero = false;
-					}
-
-					Future<Tuple2<Integer, Long>> result = getKvStateWithRetries(
-							client,
-							jobId,
-							queryName,
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							reducingState,
-							QUERY_RETRY_DELAY,
-							false);
-
-					result.onSuccess(new OnSuccess<Tuple2<Integer, Long>>() {
-						@Override
-						public void onSuccess(Tuple2<Integer, Long> result) throws Throwable {
-							counts.set(key, result.f1);
-							assertEquals("Key mismatch", key, result.f0.intValue());
-						}
-					}, testActorSystem.dispatcher());
-
-					futures.add(result);
-				}
-
-				Future<Iterable<Tuple2<Integer, Long>>> futureSequence = Futures.sequence(
-						futures,
-						testActorSystem.dispatcher());
-
-				Await.ready(futureSequence, deadline.timeLeft());
-			}
-
-			assertTrue("Not all keys are non-zero", allNonZero);
-
-			// All should be non-zero
-			for (int i = 0; i < numKeys; i++) {
-				long count = counts.get(i);
-				assertTrue("Count at position " + i + " is " + count, count > 0);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests that duplicate query registrations fail the job at the JobManager.
-	 *
-	 * <b>NOTE: </b> This test is only in the non-HA variant of the tests because
-	 * in the HA mode we use the actual JM code which does not recognize the
-	 * {@code NotifyWhenJobStatus} message.	 *
-	 */
-	@Test
-	public void testDuplicateRegistrationFailsJob() throws Exception {
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-		final int numKeys = 256;
-
-		JobID jobId = null;
-
-		try {
-			//
-			// Test program
-			//
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestKeyRangeSource(numKeys));
-
-			// Reducing state
-			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState = new ReducingStateDescriptor<>(
-					"any-name",
-					new SumReduce(),
-					source.getType());
-
-			final String queryName = "duplicate-me";
-
-			final QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = -4126824763829132959L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState(queryName, reducingState);
-
-			final QueryableStateStream<Integer, Tuple2<Integer, Long>> duplicate =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = -6265024000462809436L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState(queryName);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			Future<TestingJobManagerMessages.JobStatusIs> failedFuture = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.FAILED), deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<TestingJobManagerMessages.JobStatusIs>apply(TestingJobManagerMessages.JobStatusIs.class));
-
-			cluster.submitJobDetached(jobGraph);
-
-			TestingJobManagerMessages.JobStatusIs jobStatus = Await.result(failedFuture, deadline.timeLeft());
-			assertEquals(JobStatus.FAILED, jobStatus.state());
-
-			// Get the job and check the cause
-			JobManagerMessages.JobFound jobFound = Await.result(
-					cluster.getLeaderGateway(deadline.timeLeft())
-							.ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft())
-							.mapTo(ClassTag$.MODULE$.<JobManagerMessages.JobFound>apply(JobManagerMessages.JobFound.class)),
-					deadline.timeLeft());
-
-			String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString();
-
-			assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException"));
-			int causedByIndex = failureCause.indexOf("Caused by: ");
-			String subFailureCause = failureCause.substring(causedByIndex + "Caused by: ".length());
-			assertTrue("Not caused by IllegalStateException", subFailureCause.startsWith("java.lang.IllegalStateException"));
-			assertTrue("Exception does not contain registration name", subFailureCause.contains(queryName));
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<JobManagerMessages.CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<JobManagerMessages.CancellationSuccess>apply(JobManagerMessages.CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-		}
-	}
-
-	/**
-	 * Tests simple value state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The tests succeeds after each subtask index is queried with
-	 * value numElements (the latest element updated the state).
-	 */
-	@Test
-	public void testValueState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Value state
-			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
-					"any",
-					source.getType());
-
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 7662520075515707428L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("hakuna", valueState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-			long expected = numElements;
-
-			executeQuery(deadline, client, jobId, "hakuna", valueState, expected);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Similar tests as {@link #testValueState()} but before submitting the
-	 * job, we already issue one request which fails.
-	 */
-	@Test
-	public void testQueryNonStartedJobState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-				.addSource(new TestAscendingValueSource(numElements));
-
-			// Value state
-			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
-				"any",
-				source.getType(),
-				null);
-
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-				source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-					private static final long serialVersionUID = 7480503339992214681L;
-
-					@Override
-					public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-						return value.f0;
-					}
-				}).asQueryableState("hakuna", valueState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			// Now query
-			long expected = numElements;
-
-			// query once
-			client.getKvState(
-					jobId,
-					queryableState.getQueryableStateName(),
-					0,
-					VoidNamespace.INSTANCE,
-					BasicTypeInfo.INT_TYPE_INFO,
-					VoidNamespaceTypeInfo.INSTANCE,
-					valueState);
-
-			cluster.submitJobDetached(jobGraph);
-
-			executeQuery(deadline, client, jobId, "hakuna", valueState, expected);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Retry a query for state for keys between 0 and {@link #maxParallelism} until
-	 * <tt>expected</tt> equals the value of the result tuple's second field.
-	 */
-	private void executeQuery(
-			final Deadline deadline,
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryableStateName,
-			final StateDescriptor<?, Tuple2<Integer, Long>> stateDescriptor,
-			final long expected) throws Exception {
-
-		for (int key = 0; key < maxParallelism; key++) {
-			boolean success = false;
-			while (deadline.hasTimeLeft() && !success) {
-				Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
-					jobId,
-					queryableStateName,
-					key,
-					BasicTypeInfo.INT_TYPE_INFO,
-					stateDescriptor,
-					QUERY_RETRY_DELAY,
-					false);
-
-				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
-
-				assertEquals("Key mismatch", key, value.f0.intValue());
-				if (expected == value.f1) {
-					success = true;
-				} else {
-					// Retry
-					Thread.sleep(50);
-				}
-			}
-
-			assertTrue("Did not succeed query", success);
-		}
-	}
-
-	/**
-	 * Retry a query for state for keys between 0 and {@link #maxParallelism} until
-	 * <tt>expected</tt> equals the value of the result tuple's second field.
-	 */
-	private void executeQuery(
-			final Deadline deadline,
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryableStateName,
-			final TypeSerializer<Tuple2<Integer, Long>> valueSerializer,
-			final long expected) throws Exception {
-
-		for (int key = 0; key < maxParallelism; key++) {
-			boolean success = false;
-			while (deadline.hasTimeLeft() && !success) {
-				Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
-						jobId,
-						queryableStateName,
-						key,
-						BasicTypeInfo.INT_TYPE_INFO,
-						valueSerializer,
-						QUERY_RETRY_DELAY,
-						false);
-
-				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
-
-				assertEquals("Key mismatch", key, value.f0.intValue());
-				if (expected == value.f1) {
-					success = true;
-				} else {
-					// Retry
-					Thread.sleep(50);
-				}
-			}
-
-			assertTrue("Did not succeed query", success);
-		}
-	}
-
-	/**
-	 * Tests simple value state queryable state instance with a default value
-	 * set. Each source emits (subtaskIndex, 0)..(subtaskIndex, numElements)
-	 * tuples, the key is mapped to 1 but key 0 is queried which should throw
-	 * a {@link UnknownKeyOrNamespace} exception.
-	 *
-	 * @throws UnknownKeyOrNamespace thrown due querying a non-existent key
-	 */
-	@Test(expected = UnknownKeyOrNamespace.class)
-	public void testValueStateDefault() throws
-		Exception, UnknownKeyOrNamespace {
-
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies
-				.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-				.addSource(new TestAscendingValueSource(numElements));
-
-			// Value state
-			ValueStateDescriptor<Tuple2<Integer, Long>> valueState =
-				new ValueStateDescriptor<>(
-					"any",
-					source.getType(),
-					Tuple2.of(0, 1337L));
-
-			// only expose key "1"
-			QueryableStateStream<Integer, Tuple2<Integer, Long>>
-				queryableState =
-				source.keyBy(
-					new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 4509274556892655887L;
-
-						@Override
-						public Integer getKey(
-							Tuple2<Integer, Long> value) throws
-							Exception {
-							return 1;
-						}
-					}).asQueryableState("hakuna", valueState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-			int key = 0;
-			Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
-				jobId,
-				queryableState.getQueryableStateName(),
-				key,
-				BasicTypeInfo.INT_TYPE_INFO,
-				valueState,
-				QUERY_RETRY_DELAY,
-				true);
-
-			Await.result(future, deadline.timeLeft());
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new JobManagerMessages.CancelJob(jobId),
-						deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(
-						CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests simple value state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The tests succeeds after each subtask index is queried with
-	 * value numElements (the latest element updated the state).
-	 *
-	 * <p>This is the same as the simple value state test, but uses the API shortcut.
-	 */
-	@Test
-	public void testValueStateShortcut() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Value state shortcut
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 9168901838808830068L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("matata");
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-			long expected = numElements;
-
-			executeQuery(deadline, client, jobId, "matata",
-					queryableState.getValueSerializer(), expected);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests simple folding state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The folding state sums these up and maps them to Strings. The
-	 * test succeeds after each subtask index is queried with result n*(n+1)/2
-	 * (as a String).
-	 */
-	@Test
-	public void testFoldingState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Folding state
-			FoldingStateDescriptor<Tuple2<Integer, Long>, String> foldingState =
-					new FoldingStateDescriptor<>(
-							"any",
-							"0",
-							new SumFold(),
-							StringSerializer.INSTANCE);
-
-			QueryableStateStream<Integer, String> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = -842809958106747539L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("pumba", foldingState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Now query
-			String expected = Integer.toString(numElements * (numElements + 1) / 2);
-
-			for (int key = 0; key < maxParallelism; key++) {
-				boolean success = false;
-				while (deadline.hasTimeLeft() && !success) {
-					Future<String> future = getKvStateWithRetries(client,
-							jobId,
-							queryableState.getQueryableStateName(),
-							key,
-							BasicTypeInfo.INT_TYPE_INFO,
-							foldingState,
-							QUERY_RETRY_DELAY,
-							false);
-
-					String value = Await.result(future, deadline.timeLeft());
-					if (expected.equals(value)) {
-						success = true;
-					} else {
-						// Retry
-						Thread.sleep(50);
-					}
-				}
-
-				assertTrue("Did not succeed query", success);
-			}
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests simple reducing state queryable state instance. Each source emits
-	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
-	 * queried. The reducing state instance sums these up. The test succeeds
-	 * after each subtask index is queried with result n*(n+1)/2.
-	 */
-	@Test
-	public void testReducingState() throws Exception {
-		// Config
-		final Deadline deadline = TEST_TIMEOUT.fromNow();
-
-		final int numElements = 1024;
-
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
-
-		JobID jobId = null;
-		try {
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setStateBackend(stateBackend);
-			env.setParallelism(maxParallelism);
-			// Very important, because cluster is shared between tests and we
-			// don't explicitly check that all slots are available before
-			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
-
-			DataStream<Tuple2<Integer, Long>> source = env
-					.addSource(new TestAscendingValueSource(numElements));
-
-			// Reducing state
-			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState =
-					new ReducingStateDescriptor<>(
-							"any",
-							new SumReduce(),
-							source.getType());
-
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 8470749712274833552L;
-
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("jungle", reducingState);
-
-			// Submit the job graph
-			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-			jobId = jobGraph.getJobID();
-
-			cluster.submitJobDetached(jobGraph);
-
-			// Wait until job is running
-
-			// Now query
-			long expected = numElements * (numElements + 1) / 2;
-
-			executeQuery(deadline, client, jobId, "jungle", reducingState, expected);
-		} finally {
-			// Free cluster resources
-			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
-
-				Await.ready(cancellation, deadline.timeLeft());
-			}
-
-			client.shutDown();
-		}
-	}
-
-	private static <K, V> Future<V> getKvStateWithRetries(
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final TypeSerializer<V> valueTypeSerializer,
-			final FiniteDuration retryDelay,
-			final boolean failForUnknownKeyOrNamespace) {
-
-		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, valueTypeSerializer)
-				.recoverWith(new Recover<Future<V>>() {
-					@Override
-					public Future<V> recover(Throwable failure) throws Throwable {
-						if (failure instanceof AssertionError) {
-							return Futures.failed(failure);
-						} else if (failForUnknownKeyOrNamespace &&
-								(failure instanceof UnknownKeyOrNamespace)) {
-							return Futures.failed(failure);
-						} else {
-							// At startup some failures are expected
-							// due to races. Make sure that they don't
-							// fail this test.
-							return Patterns.after(
-									retryDelay,
-									testActorSystem.scheduler(),
-									testActorSystem.dispatcher(),
-									new Callable<Future<V>>() {
-										@Override
-										public Future<V> call() throws Exception {
-											return getKvStateWithRetries(
-													client,
-													jobId,
-													queryName,
-													key,
-													keyTypeInfo,
-													valueTypeSerializer,
-													retryDelay,
-													failForUnknownKeyOrNamespace);
-										}
-									});
-						}
-					}
-				}, testActorSystem.dispatcher());
-
-	}
-
-	private static <K, V> Future<V> getKvStateWithRetries(
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final StateDescriptor<?, V> stateDescriptor,
-			final FiniteDuration retryDelay,
-			final boolean failForUnknownKeyOrNamespace) {
-
-		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor)
-				.recoverWith(new Recover<Future<V>>() {
-					@Override
-					public Future<V> recover(Throwable failure) throws Throwable {
-						if (failure instanceof AssertionError) {
-							return Futures.failed(failure);
-						} else if (failForUnknownKeyOrNamespace &&
-								(failure instanceof UnknownKeyOrNamespace)) {
-							return Futures.failed(failure);
-						} else {
-							// At startup some failures are expected
-							// due to races. Make sure that they don't
-							// fail this test.
-							return Patterns.after(
-									retryDelay,
-									testActorSystem.scheduler(),
-									testActorSystem.dispatcher(),
-									new Callable<Future<V>>() {
-										@Override
-										public Future<V> call() throws Exception {
-											return getKvStateWithRetries(
-													client,
-													jobId,
-													queryName,
-													key,
-													keyTypeInfo,
-													stateDescriptor,
-													retryDelay,
-													failForUnknownKeyOrNamespace);
-										}
-									});
-						}
-					}
-				}, testActorSystem.dispatcher());
-	}
-
-	/**
-	 * Test source producing (key, 0)..(key, maxValue) with key being the sub
-	 * task index.
-	 *
-	 * <p>After all tuples have been emitted, the source waits to be cancelled
-	 * and does not immediately finish.
-	 */
-	private static class TestAscendingValueSource extends RichParallelSourceFunction<Tuple2<Integer, Long>> {
-
-		private static final long serialVersionUID = 1459935229498173245L;
-
-		private final long maxValue;
-		private volatile boolean isRunning = true;
-
-		TestAscendingValueSource(long maxValue) {
-			Preconditions.checkArgument(maxValue >= 0);
-			this.maxValue = maxValue;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, Long>> ctx) throws Exception {
-			// f0 => key
-			int key = getRuntimeContext().getIndexOfThisSubtask();
-			Tuple2<Integer, Long> record = new Tuple2<>(key, 0L);
-
-			long currentValue = 0;
-			while (isRunning && currentValue <= maxValue) {
-				synchronized (ctx.getCheckpointLock()) {
-					record.f1 = currentValue;
-					ctx.collect(record);
-				}
-
-				currentValue++;
-			}
-
-			while (isRunning) {
-				synchronized (this) {
-					this.wait();
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-
-			synchronized (this) {
-				this.notifyAll();
-			}
-		}
-
-	}
-
-	/**
-	 * Test source producing (key, 1) tuples with random key in key range (numKeys).
-	 */
-	protected static class TestKeyRangeSource extends RichParallelSourceFunction<Tuple2<Integer, Long>>
-			implements CheckpointListener {
-		private static final long serialVersionUID = -5744725196953582710L;
-
-		private static final AtomicLong LATEST_CHECKPOINT_ID = new AtomicLong();
-		private final int numKeys;
-		private final ThreadLocalRandom random = ThreadLocalRandom.current();
-		private volatile boolean isRunning = true;
-
-		TestKeyRangeSource(int numKeys) {
-			this.numKeys = numKeys;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-				LATEST_CHECKPOINT_ID.set(0);
-			}
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, Long>> ctx) throws Exception {
-			// f0 => key
-			Tuple2<Integer, Long> record = new Tuple2<>(0, 1L);
-
-			while (isRunning) {
-				synchronized (ctx.getCheckpointLock()) {
-					record.f0 = random.nextInt(numKeys);
-					ctx.collect(record);
-				}
-				// mild slow down
-				Thread.sleep(1);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) throws Exception {
-			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-				LATEST_CHECKPOINT_ID.set(checkpointId);
-			}
-		}
-	}
-
-	/**
-	 * Test {@link FoldFunction} concatenating the already stored string with the long passed as argument.
-	 */
-	private static class SumFold implements FoldFunction<Tuple2<Integer, Long>, String> {
-		private static final long serialVersionUID = -6249227626701264599L;
-
-		@Override
-		public String fold(String accumulator, Tuple2<Integer, Long> value) throws Exception {
-			long acc = Long.valueOf(accumulator);
-			acc += value.f1;
-			return Long.toString(acc);
-		}
-	}
-
-	/**
-	 * Test {@link ReduceFunction} summing up its two arguments.
-	 */
-	protected static class SumReduce implements ReduceFunction<Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = -8651235077342052336L;
-
-		@Override
-		public Tuple2<Integer, Long> reduce(Tuple2<Integer, Long> value1, Tuple2<Integer, Long> value2) throws Exception {
-			value1.f1 += value2.f1;
-			return value1;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/HAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/HAAbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/HAAbstractQueryableStateITCase.java
deleted file mode 100644
index cd89e00..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/HAAbstractQueryableStateITCase.java
+++ /dev/null
@@ -1,102 +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.test.query;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.QueryableStateOptions;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.testingUtils.TestingCluster;
-
-import org.apache.curator.test.TestingServer;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.rules.TemporaryFolder;
-
-import static org.junit.Assert.fail;
-
-/**
- * Base class with the cluster configuration for the tests on the NON-HA mode.
- */
-public abstract class HAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
-
-	private static final int NUM_JMS = 2;
-	private static final int NUM_TMS = 4;
-	private static final int NUM_SLOTS_PER_TM = 4;
-
-	private static TestingServer zkServer;
-	private static TemporaryFolder temporaryFolder;
-
-	@BeforeClass
-	public static void setup() {
-		try {
-			zkServer = new TestingServer();
-			temporaryFolder = new TemporaryFolder();
-			temporaryFolder.create();
-
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS);
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
-			config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true);
-			config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2);
-			config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2);
-			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString());
-			config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString());
-			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
-
-			cluster = new TestingCluster(config, false);
-			cluster.start();
-
-			testActorSystem = AkkaUtils.createDefaultActorSystem();
-
-			// verify that we are in HA mode
-			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.ZOOKEEPER);
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDown() {
-		if (cluster != null) {
-			cluster.stop();
-			cluster.awaitTermination();
-		}
-
-		testActorSystem.shutdown();
-		testActorSystem.awaitTermination();
-
-		try {
-			zkServer.stop();
-			zkServer.close();
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-
-		temporaryFolder.delete();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseFsBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseFsBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseFsBackend.java
deleted file mode 100644
index 5d5b671..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseFsBackend.java
+++ /dev/null
@@ -1,39 +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.test.query;
-
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link FsStateBackend}.
- */
-public class HAQueryableStateITCaseFsBackend extends HAAbstractQueryableStateITCase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseRocksDBBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseRocksDBBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseRocksDBBackend.java
deleted file mode 100644
index 22570b5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/HAQueryableStateITCaseRocksDBBackend.java
+++ /dev/null
@@ -1,39 +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.test.query;
-
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
- */
-public class HAQueryableStateITCaseRocksDBBackend extends HAAbstractQueryableStateITCase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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
deleted file mode 100644
index 0c628e4..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
+++ /dev/null
@@ -1,168 +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.test.query;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.contrib.streaming.state.PredefinedOptions;
-import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
-import org.apache.flink.runtime.query.TaskKvStateRegistry;
-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.apache.flink.runtime.state.internal.InternalMapState;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.rocksdb.ColumnFamilyOptions;
-import org.rocksdb.DBOptions;
-
-import java.io.File;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Additional tests for the serialization and deserialization of {@link
- * org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer}
- * with a RocksDB state back-end.
- */
-public final class KVStateRequestSerializerRocksDBTest {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	/**
-	 * Extension of {@link RocksDBKeyedStateBackend} to make {@link
-	 * #createListState(TypeSerializer, ListStateDescriptor)} public for use in
-	 * the tests.
-	 *
-	 * @param <K> key type
-	 */
-	static final class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {
-
-		RocksDBKeyedStateBackend2(
-				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,
-				final ExecutionConfig executionConfig) throws Exception {
-
-			super(operatorIdentifier, userCodeClassLoader,
-				instanceBasePath,
-				dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer,
-				numberOfKeyGroups, keyGroupRange, executionConfig, false);
-		}
-
-		@Override
-		public <N, T> InternalListState<N, T> createListState(
-			final TypeSerializer<N> namespaceSerializer,
-			final ListStateDescriptor<T> stateDesc) throws Exception {
-
-			return super.createListState(namespaceSerializer, stateDesc);
-		}
-	}
-
-	/**
-	 * Tests list serialization and deserialization match.
-	 *
-	 * @see KvStateRequestSerializerTest#testListSerialization()
-	 * KvStateRequestSerializerTest#testListSerialization() using the heap state back-end
-	 * test
-	 */
-	@Test
-	public void testListSerialization() throws Exception {
-		final long key = 0L;
-
-		// objects for RocksDB state list serialisation
-		DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
-		dbOptions.setCreateIfMissing(true);
-		ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
-		final RocksDBKeyedStateBackend2<Long> longHeapKeyedStateBackend =
-			new RocksDBKeyedStateBackend2<>(
-				"no-op",
-				ClassLoader.getSystemClassLoader(),
-				temporaryFolder.getRoot(),
-				dbOptions,
-				columnFamilyOptions,
-				mock(TaskKvStateRegistry.class),
-				LongSerializer.INSTANCE,
-				1, new KeyGroupRange(0, 0),
-				new ExecutionConfig()
-			);
-		longHeapKeyedStateBackend.restore(null);
-		longHeapKeyedStateBackend.setCurrentKey(key);
-
-		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend
-			.createListState(VoidNamespaceSerializer.INSTANCE,
-				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
-
-		KvStateRequestSerializerTest.testListSerialization(key, listState);
-	}
-
-	/**
-	 * Tests map serialization and deserialization match.
-	 *
-	 * @see KvStateRequestSerializerTest#testMapSerialization()
-	 * KvStateRequestSerializerTest#testMapSerialization() using the heap state back-end
-	 * test
-	 */
-	@Test
-	public void testMapSerialization() throws Exception {
-		final long key = 0L;
-
-		// objects for RocksDB state list serialisation
-		DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
-		dbOptions.setCreateIfMissing(true);
-		ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
-		final RocksDBKeyedStateBackend<Long> longHeapKeyedStateBackend =
-			new RocksDBKeyedStateBackend<>(
-				"no-op",
-				ClassLoader.getSystemClassLoader(),
-				temporaryFolder.getRoot(),
-				dbOptions,
-				columnFamilyOptions,
-				mock(TaskKvStateRegistry.class),
-				LongSerializer.INSTANCE,
-				1, new KeyGroupRange(0, 0),
-				new ExecutionConfig(),
-				false);
-		longHeapKeyedStateBackend.restore(null);
-		longHeapKeyedStateBackend.setCurrentKey(key);
-
-		final InternalMapState<VoidNamespace, Long, String> mapState = (InternalMapState<VoidNamespace, Long, String>)
-				longHeapKeyedStateBackend.getPartitionedState(
-						VoidNamespace.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE,
-						new MapStateDescriptor<>("test", LongSerializer.INSTANCE, StringSerializer.INSTANCE));
-
-		KvStateRequestSerializerTest.testMapSerialization(key, mapState);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/NonHAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAAbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/NonHAAbstractQueryableStateITCase.java
deleted file mode 100644
index 83f86e4..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAAbstractQueryableStateITCase.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.query;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.QueryableStateOptions;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.testingUtils.TestingCluster;
-
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-import static org.junit.Assert.fail;
-
-/**
- * Base class with the cluster configuration for the tests on the HA mode.
- */
-public abstract class NonHAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
-
-	private static final int NUM_TMS = 2;
-	private static final int NUM_SLOTS_PER_TM = 4;
-
-	@BeforeClass
-	public static void setup() {
-		try {
-			Configuration config = new Configuration();
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
-			config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1);
-			config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true);
-			config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1);
-
-			cluster = new TestingCluster(config, false);
-			cluster.start(true);
-
-			testActorSystem = AkkaUtils.createDefaultActorSystem();
-
-			// verify that we are not in HA mode
-			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.NONE);
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDown() {
-		try {
-			cluster.shutdown();
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseFsBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseFsBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseFsBackend.java
deleted file mode 100644
index d4dbe83..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseFsBackend.java
+++ /dev/null
@@ -1,39 +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.test.query;
-
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link FsStateBackend}.
- */
-public class NonHAQueryableStateITCaseFsBackend extends NonHAAbstractQueryableStateITCase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseRocksDBBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseRocksDBBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseRocksDBBackend.java
deleted file mode 100644
index a15e6a4..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/query/NonHAQueryableStateITCaseRocksDBBackend.java
+++ /dev/null
@@ -1,39 +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.test.query;
-
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-
-import org.junit.Rule;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
- */
-public class NonHAQueryableStateITCaseRocksDBBackend extends NonHAAbstractQueryableStateITCase {
-
-	@Rule
-	public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-	@Override
-	protected AbstractStateBackend createStateBackend() throws Exception {
-		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 524e718..1bb3732 100644
--- a/pom.xml
+++ b/pom.xml
@@ -81,6 +81,7 @@ under the License.
 		<module>flink-yarn</module>
 		<module>flink-yarn-tests</module>
 		<module>flink-fs-tests</module>
+		<module>flink-queryable-state</module>
 	</modules>
 
 	<properties>


[04/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
deleted file mode 100644
index 4ba6929..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateClient.java
+++ /dev/null
@@ -1,587 +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.query;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.ListSerializer;
-import org.apache.flink.configuration.AkkaOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.configuration.QueryableStateOptions;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.FixedDelayLookupRetryStrategyFactory;
-import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.LookupRetryStrategyFactory;
-import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.KvStateClient;
-import org.apache.flink.runtime.query.netty.KvStateServer;
-import org.apache.flink.runtime.query.netty.UnknownKeyOrNamespace;
-import org.apache.flink.runtime.query.netty.UnknownKvStateID;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceTypeInfo;
-import org.apache.flink.util.Preconditions;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-import akka.dispatch.Recover;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.ConnectException;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
-
-/**
- * Client for queryable state.
- *
- * <p>You can mark state as queryable via {@link StateDescriptor#setQueryable(String)}.
- * The state instance created from this descriptor will be published for queries
- * when it's created on the TaskManagers and the location will be reported to
- * the JobManager.
- *
- * <p>The client resolves the location of the requested KvState via the
- * JobManager. Resolved locations are cached. When the server address of the
- * requested KvState instance is determined, the client sends out a request to
- * the server.
- */
-public class QueryableStateClient {
-
-	private static final Logger LOG = LoggerFactory.getLogger(QueryableStateClient.class);
-
-	/**
-	 * {@link KvStateLocation} lookup to resolve the address of KvState instances.
-	 */
-	private final KvStateLocationLookupService lookupService;
-
-	/**
-	 * Network client for queries against {@link KvStateServer} instances.
-	 */
-	private final KvStateClient kvStateClient;
-
-	/**
-	 * Execution context.
-	 */
-	private final ExecutionContext executionContext;
-
-	/**
-	 * Cache for {@link KvStateLocation} instances keyed by job and name.
-	 */
-	private final ConcurrentMap<Tuple2<JobID, String>, Future<KvStateLocation>> lookupCache =
-			new ConcurrentHashMap<>();
-
-	/** This is != null, if we started the actor system. */
-	private final ActorSystem actorSystem;
-
-	private ExecutionConfig executionConfig;
-
-	/**
-	 * Creates a client from the given configuration.
-	 *
-	 * <p>This will create multiple Thread pools: one for the started actor
-	 * system and another for the network client.
-	 *
-	 * @param config Configuration to use.
-	 * @throws Exception Failures are forwarded
-	 */
-	public QueryableStateClient(Configuration config) throws Exception {
-		this(config, HighAvailabilityServicesUtils.createHighAvailabilityServices(
-				config, Executors.directExecutor(), HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION));
-	}
-
-	/**
-	 * Creates a client from the given configuration.
-	 *
-	 * <p>This will create multiple Thread pools: one for the started actor
-	 * system and another for the network client.
-	 *
-	 * @param config Configuration to use.
-	 * @param highAvailabilityServices Service factory for high availability services
-	 * @throws Exception Failures are forwarded
-	 *
-	 * @deprecated This constructor is deprecated and stays only for backwards compatibility. Use the
-	 * {@link #QueryableStateClient(Configuration)} instead.
-	 */
-	@Deprecated
-	public QueryableStateClient(
-			Configuration config,
-			HighAvailabilityServices highAvailabilityServices) throws Exception {
-		Preconditions.checkNotNull(config, "Configuration");
-
-		// Create a leader retrieval service
-		LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
-
-		// Get the ask timeout
-		String askTimeoutString = config.getString(AkkaOptions.ASK_TIMEOUT);
-
-		Duration timeout = FiniteDuration.apply(askTimeoutString);
-		if (!timeout.isFinite()) {
-			throw new IllegalConfigurationException(AkkaOptions.ASK_TIMEOUT.key()
-					+ " is not a finite timeout ('" + askTimeoutString + "')");
-		}
-
-		FiniteDuration askTimeout = (FiniteDuration) timeout;
-
-		int lookupRetries = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRIES);
-		int lookupRetryDelayMillis = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRY_DELAY);
-
-		// Retries if no JobManager is around
-		LookupRetryStrategyFactory retryStrategy = new FixedDelayLookupRetryStrategyFactory(
-				lookupRetries,
-				FiniteDuration.apply(lookupRetryDelayMillis, "ms"));
-
-		// Create the actor system
-		@SuppressWarnings("unchecked")
-		Option<Tuple2<String, Object>> remoting = new Some(new Tuple2<>("", 0));
-		this.actorSystem = AkkaUtils.createActorSystem(config, remoting);
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				actorSystem,
-				askTimeout,
-				retryStrategy);
-
-		int numEventLoopThreads = config.getInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS);
-
-		if (numEventLoopThreads == 0) {
-			numEventLoopThreads = Runtime.getRuntime().availableProcessors();
-		}
-
-		// Create the network client
-		KvStateClient networkClient = new KvStateClient(
-				numEventLoopThreads,
-				new DisabledKvStateRequestStats());
-
-		this.lookupService = lookupService;
-		this.kvStateClient = networkClient;
-		this.executionContext = actorSystem.dispatcher();
-		this.executionConfig = new ExecutionConfig();
-
-		this.lookupService.start();
-	}
-
-	/** Gets the {@link ExecutionConfig}. */
-	public ExecutionConfig getExecutionConfig() {
-		return executionConfig;
-	}
-
-	/** Sets the {@link ExecutionConfig}. */
-	public void setExecutionConfig(ExecutionConfig config) {
-		this.executionConfig = config;
-	}
-
-	/**
-	 * Creates a client.
-	 *
-	 * @param lookupService    Location lookup service
-	 * @param kvStateClient    Network client for queries
-	 * @param executionContext Execution context for futures
-	 */
-	public QueryableStateClient(
-			KvStateLocationLookupService lookupService,
-			KvStateClient kvStateClient,
-			ExecutionContext executionContext) {
-
-		this.lookupService = Preconditions.checkNotNull(lookupService, "KvStateLocationLookupService");
-		this.kvStateClient = Preconditions.checkNotNull(kvStateClient, "KvStateClient");
-		this.executionContext = Preconditions.checkNotNull(executionContext, "ExecutionContext");
-		this.actorSystem = null;
-
-		this.lookupService.start();
-	}
-
-	/**
-	 * Returns the execution context of this client.
-	 *
-	 * @return The execution context used by the client.
-	 */
-	public ExecutionContext getExecutionContext() {
-		return executionContext;
-	}
-
-	/**
-	 * Shuts down the client and all components.
-	 */
-	public void shutDown() {
-		try {
-			lookupService.shutDown();
-		} catch (Throwable t) {
-			LOG.error("Failed to shut down KvStateLookupService", t);
-		}
-
-		try {
-			kvStateClient.shutDown();
-		} catch (Throwable t) {
-			LOG.error("Failed to shut down KvStateClient", t);
-		}
-
-		if (actorSystem != null) {
-			try {
-				actorSystem.shutdown();
-			} catch (Throwable t) {
-				LOG.error("Failed to shut down ActorSystem", t);
-			}
-		}
-	}
-
-	/**
-	 * Returns a future holding the serialized request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state
-	 *                                  belongs to
-	 * @param queryableStateName        Name under which the state is queryable
-	 * @param keyHashCode               Integer hash code of the key (result of
-	 *                                  a call to {@link Object#hashCode()}
-	 * @param serializedKeyAndNamespace Serialized key and namespace to query
-	 *                                  KvState instance with
-	 * @return Future holding the serialized result
-	 */
-	@SuppressWarnings("unchecked")
-	public Future<byte[]> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final int keyHashCode,
-			final byte[] serializedKeyAndNamespace) {
-
-		return getKvState(jobId, queryableStateName, keyHashCode, serializedKeyAndNamespace, false)
-				.recoverWith(new Recover<Future<byte[]>>() {
-					@Override
-					public Future<byte[]> recover(Throwable failure) throws Throwable {
-						if (failure instanceof UnknownKvStateID ||
-								failure instanceof UnknownKvStateKeyGroupLocation ||
-								failure instanceof UnknownKvStateLocation ||
-								failure instanceof ConnectException) {
-							// These failures are likely to be caused by out-of-sync
-							// KvStateLocation. Therefore we retry this query and
-							// force look up the location.
-							return getKvState(
-									jobId,
-									queryableStateName,
-									keyHashCode,
-									serializedKeyAndNamespace,
-									true);
-						} else {
-							return Futures.failed(failure);
-						}
-					}
-				}, executionContext);
-	}
-
-	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state belongs to.
-	 * @param queryableStateName        Name under which the state is queryable.
-	 * @param key			            The key we are interested in.
-	 * @param keyTypeHint				A {@link TypeHint} used to extract the type of the key.
-	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
-	 * @return Future holding the result.
-	 */
-	@PublicEvolving
-	public <K, V> Future<V> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final TypeHint<K> keyTypeHint,
-			final StateDescriptor<?, V> stateDescriptor) {
-
-		Preconditions.checkNotNull(keyTypeHint);
-
-		TypeInformation<K> keyTypeInfo = keyTypeHint.getTypeInfo();
-		return getKvState(jobId, queryableStateName, key, keyTypeInfo, stateDescriptor);
-	}
-
-	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state belongs to.
-	 * @param queryableStateName        Name under which the state is queryable.
-	 * @param key			            The key we are interested in.
-	 * @param keyTypeInfo				The {@link TypeInformation} of the key.
-	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
-	 * @return Future holding the result.
-	 */
-	@PublicEvolving
-	public <K, V> Future<V> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final StateDescriptor<?, V> stateDescriptor) {
-
-		Preconditions.checkNotNull(keyTypeInfo);
-
-		return getKvState(jobId, queryableStateName, key, VoidNamespace.INSTANCE,
-				keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor);
-	}
-
-	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state belongs to.
-	 * @param queryableStateName        Name under which the state is queryable.
-	 * @param key			            The key that the state we request is associated with.
-	 * @param namespace					The namespace of the state.
-	 * @param keyTypeInfo				The {@link TypeInformation} of the keys.
-	 * @param namespaceTypeInfo			The {@link TypeInformation} of the namespace.
-	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
-	 * @return Future holding the result.
-	 */
-	@PublicEvolving
-	public <K, V, N> Future<V> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final N namespace,
-			final TypeInformation<K> keyTypeInfo,
-			final TypeInformation<N> namespaceTypeInfo,
-			final StateDescriptor<?, V> stateDescriptor) {
-
-		Preconditions.checkNotNull(stateDescriptor);
-
-		// initialize the value serializer based on the execution config.
-		stateDescriptor.initializeSerializerUnlessSet(executionConfig);
-		TypeSerializer<V> stateSerializer = stateDescriptor.getSerializer();
-
-		return getKvState(jobId, queryableStateName, key,
-				namespace, keyTypeInfo, namespaceTypeInfo, stateSerializer);
-	}
-
-	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state belongs to.
-	 * @param queryableStateName        Name under which the state is queryable.
-	 * @param key			            The key that the state we request is associated with.
-	 * @param namespace					The namespace of the state.
-	 * @param keyTypeInfo				The {@link TypeInformation} of the keys.
-	 * @param namespaceTypeInfo			The {@link TypeInformation} of the namespace.
-	 * @param stateSerializer			The {@link TypeSerializer} of the state we want to query.
-	 * @return Future holding the result.
-	 */
-	@PublicEvolving
-	public <K, V, N> Future<V> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final K key,
-			final N namespace,
-			final TypeInformation<K> keyTypeInfo,
-			final TypeInformation<N> namespaceTypeInfo,
-			final TypeSerializer<V> stateSerializer) {
-
-		Preconditions.checkNotNull(queryableStateName);
-
-		Preconditions.checkNotNull(key);
-		Preconditions.checkNotNull(namespace);
-
-		Preconditions.checkNotNull(keyTypeInfo);
-		Preconditions.checkNotNull(namespaceTypeInfo);
-		Preconditions.checkNotNull(stateSerializer);
-
-		if (stateSerializer instanceof ListSerializer) {
-			throw new IllegalArgumentException("ListState is not supported out-of-the-box yet.");
-		}
-
-		TypeSerializer<K> keySerializer = keyTypeInfo.createSerializer(executionConfig);
-		TypeSerializer<N> namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig);
-
-		final byte[] serializedKeyAndNamespace;
-		try {
-			serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-					key,
-					keySerializer,
-					namespace,
-					namespaceSerializer);
-		} catch (IOException e) {
-			return Futures.failed(e);
-		}
-
-		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace)
-				.flatMap(new Mapper<byte[], Future<V>>() {
-					@Override
-					public Future<V> apply(byte[] parameter) {
-						try {
-							return Futures.successful(
-									KvStateRequestSerializer.deserializeValue(parameter, stateSerializer));
-						} catch (IOException e) {
-							return Futures.failed(e);
-						}
-					}
-				}, executionContext);
-	}
-
-	/**
-	 * Returns a future holding the serialized request result.
-	 *
-	 * @param jobId                     JobID of the job the queryable state
-	 *                                  belongs to
-	 * @param queryableStateName        Name under which the state is queryable
-	 * @param keyHashCode               Integer hash code of the key (result of
-	 *                                  a call to {@link Object#hashCode()}
-	 * @param serializedKeyAndNamespace Serialized key and namespace to query
-	 *                                  KvState instance with
-	 * @param forceLookup               Flag to force lookup of the {@link KvStateLocation}
-	 * @return Future holding the serialized result
-	 */
-	private Future<byte[]> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final int keyHashCode,
-			final byte[] serializedKeyAndNamespace,
-			boolean forceLookup) {
-
-		return getKvStateLookupInfo(jobId, queryableStateName, forceLookup)
-				.flatMap(new Mapper<KvStateLocation, Future<byte[]>>() {
-					@Override
-					public Future<byte[]> apply(KvStateLocation lookup) {
-						int keyGroupIndex = KeyGroupRangeAssignment.computeKeyGroupForKeyHash(keyHashCode, lookup.getNumKeyGroups());
-
-						KvStateServerAddress serverAddress = lookup.getKvStateServerAddress(keyGroupIndex);
-						if (serverAddress == null) {
-							return Futures.failed(new UnknownKvStateKeyGroupLocation());
-						} else {
-							// Query server
-							KvStateID kvStateId = lookup.getKvStateID(keyGroupIndex);
-							return kvStateClient.getKvState(serverAddress, kvStateId, serializedKeyAndNamespace);
-						}
-					}
-				}, executionContext);
-	}
-
-	/**
-	 * Lookup the {@link KvStateLocation} for the given job and queryable state
-	 * name.
-	 *
-	 * <p>The job manager will be queried for the location only if forced or no
-	 * cached location can be found. There are no guarantees about
-	 *
-	 * @param jobId              JobID the state instance belongs to.
-	 * @param queryableStateName Name under which the state instance has been published.
-	 * @param forceUpdate        Flag to indicate whether to force a update via the lookup service.
-	 * @return Future holding the KvStateLocation
-	 */
-	private Future<KvStateLocation> getKvStateLookupInfo(
-			JobID jobId,
-			final String queryableStateName,
-			boolean forceUpdate) {
-
-		if (forceUpdate) {
-			Future<KvStateLocation> lookupFuture = lookupService
-					.getKvStateLookupInfo(jobId, queryableStateName);
-			lookupCache.put(new Tuple2<>(jobId, queryableStateName), lookupFuture);
-			return lookupFuture;
-		} else {
-			Tuple2<JobID, String> cacheKey = new Tuple2<>(jobId, queryableStateName);
-			final Future<KvStateLocation> cachedFuture = lookupCache.get(cacheKey);
-
-			if (cachedFuture == null) {
-				Future<KvStateLocation> lookupFuture = lookupService
-						.getKvStateLookupInfo(jobId, queryableStateName);
-
-				Future<KvStateLocation> previous = lookupCache.putIfAbsent(cacheKey, lookupFuture);
-				if (previous == null) {
-					return lookupFuture;
-				} else {
-					return previous;
-				}
-			} else {
-				// do not retain futures which failed as they will remain in
-				// the cache even if the error cause is not present any more
-				// and a new lookup may succeed
-				if (cachedFuture.isCompleted() &&
-					cachedFuture.value().get().isFailure()) {
-					// issue a new lookup
-					Future<KvStateLocation> lookupFuture = lookupService
-						.getKvStateLookupInfo(jobId, queryableStateName);
-
-					// replace the existing one if it has not been replaced yet
-					// otherwise return the one in the cache
-					if (lookupCache.replace(cacheKey, cachedFuture, lookupFuture)) {
-						return lookupFuture;
-					} else {
-						return lookupCache.get(cacheKey);
-					}
-				} else {
-					return cachedFuture;
-				}
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
new file mode 100644
index 0000000..852d394
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
@@ -0,0 +1,89 @@
+/*
+ * 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.query;
+
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+
+/**
+ * Utility class to initialize entities used in queryable state.
+ */
+public final class QueryableStateUtils {
+
+	private static final Logger LOG = LoggerFactory.getLogger(QueryableStateUtils.class);
+
+	/**
+	 * Initializes the {@link KvStateServer server} responsible for sending the
+	 * requested internal state to the Queryable State Client.
+	 *
+	 * @param address the address to bind to.
+	 * @param port the port to listen to.
+	 * @param eventLoopThreads the number of threads to be used to process incoming requests.
+	 * @param queryThreads the number of threads to be used to send the actual state.
+	 * @param kvStateRegistry the registry with the queryable state.
+	 * @param stats statistics to be gathered about the incoming requests.
+	 * @return the {@link KvStateServer state server}.
+	 */
+	public static KvStateServer createKvStateServer(
+			final InetAddress address,
+			final int port,
+			final int eventLoopThreads,
+			final int queryThreads,
+			final KvStateRegistry kvStateRegistry,
+			final KvStateRequestStats stats) {
+
+		Preconditions.checkNotNull(address, "address");
+		Preconditions.checkNotNull(kvStateRegistry, "registry");
+		Preconditions.checkNotNull(stats, "stats");
+
+		Preconditions.checkArgument(eventLoopThreads >= 1);
+		Preconditions.checkArgument(queryThreads >= 1);
+
+		try {
+			String classname = "org.apache.flink.queryablestate.server.KvStateServerImpl";
+			Class<? extends KvStateServer> clazz = Class.forName(classname).asSubclass(KvStateServer.class);
+			Constructor<? extends KvStateServer> constructor = clazz.getConstructor(
+					InetAddress.class,
+					Integer.class,
+					Integer.class,
+					Integer.class,
+					KvStateRegistry.class,
+					KvStateRequestStats.class);
+			return constructor.newInstance(address, port, eventLoopThreads, queryThreads, kvStateRegistry, stats);
+		} catch (ClassNotFoundException e) {
+			LOG.info("Could not load Queryable State Server. " +
+					"Probable reason: flink-queryable-state is not in the classpath");
+			LOG.debug("Caught exception", e);
+			return null;
+		} catch (InvocationTargetException e) {
+			LOG.error("Queryable State Server could not be created", e.getTargetException());
+			return null;
+		} catch (Throwable t) {
+			LOG.error("Failed to instantiate the Queryable State Server.", t);
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownJobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownJobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownJobManager.java
deleted file mode 100644
index 3549ed6..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownJobManager.java
+++ /dev/null
@@ -1,33 +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.query;
-
-/**
- * Exception to fail Future with if no JobManager is currently registered at
- * the {@link KvStateLocationLookupService}.
- */
-class UnknownJobManager extends Exception {
-
-	private static final long serialVersionUID = 1L;
-
-	public UnknownJobManager() {
-		super("Unknown JobManager. Either the JobManager has not registered yet " +
-				"or has lost leadership.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownKvStateKeyGroupLocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownKvStateKeyGroupLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownKvStateKeyGroupLocation.java
deleted file mode 100644
index 8f62be5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/UnknownKvStateKeyGroupLocation.java
+++ /dev/null
@@ -1,29 +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.query;
-
-/**
- * Exception thrown if there is no location information available for the given
- * key group in a {@link KvStateLocation} instance.
- */
-class UnknownKvStateKeyGroupLocation extends Exception {
-
-	private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
deleted file mode 100644
index e4fa809..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/ChunkedByteBuf.java
+++ /dev/null
@@ -1,98 +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.query.netty;
-
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedInput;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
-/**
- * A {@link ByteBuf} instance to be consumed in chunks by {@link ChunkedWriteHandler},
- * respecting the high and low watermarks.
- *
- * @see <a href="http://normanmaurer.me/presentations/2014-facebook-eng-netty/slides.html#10.0">Low/High Watermarks</a>
- */
-class ChunkedByteBuf implements ChunkedInput<ByteBuf> {
-
-	/** The buffer to chunk. */
-	private final ByteBuf buf;
-
-	/** Size of chunks. */
-	private final int chunkSize;
-
-	/** Closed flag. */
-	private boolean isClosed;
-
-	/** End of input flag. */
-	private boolean isEndOfInput;
-
-	public ChunkedByteBuf(ByteBuf buf, int chunkSize) {
-		this.buf = Preconditions.checkNotNull(buf, "Buffer");
-		Preconditions.checkArgument(chunkSize > 0, "Non-positive chunk size");
-		this.chunkSize = chunkSize;
-	}
-
-	@Override
-	public boolean isEndOfInput() throws Exception {
-		return isClosed || isEndOfInput;
-	}
-
-	@Override
-	public void close() throws Exception {
-		if (!isClosed) {
-			// If we did not consume the whole buffer yet, we have to release
-			// it here. Otherwise, it's the responsibility of the consumer.
-			if (!isEndOfInput) {
-				buf.release();
-			}
-
-			isClosed = true;
-		}
-	}
-
-	@Override
-	public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception {
-		if (isClosed) {
-			return null;
-		} else if (buf.readableBytes() <= chunkSize) {
-			isEndOfInput = true;
-
-			// Don't retain as the consumer is responsible to release it
-			return buf.slice();
-		} else {
-			// Return a chunk sized slice of the buffer. The ref count is
-			// shared with the original buffer. That's why we need to retain
-			// a reference here.
-			return buf.readSlice(chunkSize).retain();
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "ChunkedByteBuf{" +
-				"buf=" + buf +
-				", chunkSize=" + chunkSize +
-				", isClosed=" + isClosed +
-				", isEndOfInput=" + isEndOfInput +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
deleted file mode 100644
index 1a84e83..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java
+++ /dev/null
@@ -1,579 +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.query.netty;
-
-import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
-import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
-import akka.dispatch.Futures;
-
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayDeque;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-
-/**
- * Netty-based client querying {@link KvStateServer} instances.
- *
- * <p>This client can be used by multiple threads concurrently. Operations are
- * executed asynchronously and return Futures to their result.
- *
- * <p>The incoming pipeline looks as follows:
- * <pre>
- * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
- * </pre>
- *
- * <p>Received binary messages are expected to contain a frame length field. Netty's
- * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
- * giving it to our {@link KvStateClientHandler}.
- *
- * <p>Connections are established and closed by the client. The server only
- * closes the connection on a fatal failure that cannot be recovered.
- */
-public class KvStateClient {
-
-	/** Netty's Bootstrap. */
-	private final Bootstrap bootstrap;
-
-	/** Statistics tracker. */
-	private final KvStateRequestStats stats;
-
-	/** Established connections. */
-	private final ConcurrentHashMap<KvStateServerAddress, EstablishedConnection> establishedConnections =
-			new ConcurrentHashMap<>();
-
-	/** Pending connections. */
-	private final ConcurrentHashMap<KvStateServerAddress, PendingConnection> pendingConnections =
-			new ConcurrentHashMap<>();
-
-	/** Atomic shut down flag. */
-	private final AtomicBoolean shutDown = new AtomicBoolean();
-
-	/**
-	 * Creates a client with the specified number of event loop threads.
-	 *
-	 * @param numEventLoopThreads Number of event loop threads (minimum 1).
-	 */
-	public KvStateClient(int numEventLoopThreads, KvStateRequestStats stats) {
-		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
-		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
-
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateClient Event Loop Thread %d")
-				.build();
-
-		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
-
-		this.bootstrap = new Bootstrap()
-				.group(nioGroup)
-				.channel(NioSocketChannel.class)
-				.option(ChannelOption.ALLOCATOR, bufferPool)
-				.handler(new ChannelInitializer<SocketChannel>() {
-					@Override
-					protected void initChannel(SocketChannel ch) throws Exception {
-						ch.pipeline()
-								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-								// ChunkedWriteHandler respects Channel writability
-								.addLast(new ChunkedWriteHandler());
-					}
-				});
-
-		this.stats = Preconditions.checkNotNull(stats, "Statistics tracker");
-	}
-
-	/**
-	 * Returns a future holding the serialized request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param serverAddress Address of the server to query
-	 * @param kvStateId ID of the KvState instance to query
-	 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance with
-	 * @return Future holding the serialized result
-	 */
-	public Future<byte[]> getKvState(
-			KvStateServerAddress serverAddress,
-			KvStateID kvStateId,
-			byte[] serializedKeyAndNamespace) {
-
-		if (shutDown.get()) {
-			return Futures.failed(new IllegalStateException("Shut down"));
-		}
-
-		EstablishedConnection connection = establishedConnections.get(serverAddress);
-
-		if (connection != null) {
-			return connection.getKvState(kvStateId, serializedKeyAndNamespace);
-		} else {
-			PendingConnection pendingConnection = pendingConnections.get(serverAddress);
-			if (pendingConnection != null) {
-				// There was a race, use the existing pending connection.
-				return pendingConnection.getKvState(kvStateId, serializedKeyAndNamespace);
-			} else {
-				// We try to connect to the server.
-				PendingConnection pending = new PendingConnection(serverAddress);
-				PendingConnection previous = pendingConnections.putIfAbsent(serverAddress, pending);
-
-				if (previous == null) {
-					// OK, we are responsible to connect.
-					bootstrap.connect(serverAddress.getHost(), serverAddress.getPort())
-							.addListener(pending);
-
-					return pending.getKvState(kvStateId, serializedKeyAndNamespace);
-				} else {
-					// There was a race, use the existing pending connection.
-					return previous.getKvState(kvStateId, serializedKeyAndNamespace);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Shuts down the client and closes all connections.
-	 *
-	 * <p>After a call to this method, all returned futures will be failed.
-	 */
-	public void shutDown() {
-		if (shutDown.compareAndSet(false, true)) {
-			for (Map.Entry<KvStateServerAddress, EstablishedConnection> conn : establishedConnections.entrySet()) {
-				if (establishedConnections.remove(conn.getKey(), conn.getValue())) {
-					conn.getValue().close();
-				}
-			}
-
-			for (Map.Entry<KvStateServerAddress, PendingConnection> conn : pendingConnections.entrySet()) {
-				if (pendingConnections.remove(conn.getKey()) != null) {
-					conn.getValue().close();
-				}
-			}
-
-			if (bootstrap != null) {
-				EventLoopGroup group = bootstrap.group();
-				if (group != null) {
-					group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Closes the connection to the given server address if it exists.
-	 *
-	 * <p>If there is a request to the server a new connection will be established.
-	 *
-	 * @param serverAddress Target address of the connection to close
-	 */
-	public void closeConnection(KvStateServerAddress serverAddress) {
-		PendingConnection pending = pendingConnections.get(serverAddress);
-		if (pending != null) {
-			pending.close();
-		}
-
-		EstablishedConnection established = establishedConnections.remove(serverAddress);
-		if (established != null) {
-			established.close();
-		}
-	}
-
-	/**
-	 * A pending connection that is in the process of connecting.
-	 */
-	private class PendingConnection implements ChannelFutureListener {
-
-		/** Lock to guard the connect call, channel hand in, etc. */
-		private final Object connectLock = new Object();
-
-		/** Address of the server we are connecting to. */
-		private final KvStateServerAddress serverAddress;
-
-		/** Queue of requests while connecting. */
-		private final ArrayDeque<PendingRequest> queuedRequests = new ArrayDeque<>();
-
-		/** The established connection after the connect succeeds. */
-		private EstablishedConnection established;
-
-		/** Closed flag. */
-		private boolean closed;
-
-		/** Failure cause if something goes wrong. */
-		private Throwable failureCause;
-
-		/**
-		 * Creates a pending connection to the given server.
-		 *
-		 * @param serverAddress Address of the server to connect to.
-		 */
-		private PendingConnection(KvStateServerAddress serverAddress) {
-			this.serverAddress = serverAddress;
-		}
-
-		@Override
-		public void operationComplete(ChannelFuture future) throws Exception {
-			// Callback from the Bootstrap's connect call.
-			if (future.isSuccess()) {
-				handInChannel(future.channel());
-			} else {
-				close(future.cause());
-			}
-		}
-
-		/**
-		 * Returns a future holding the serialized request result.
-		 *
-		 * <p>If the channel has been established, forward the call to the
-		 * established channel, otherwise queue it for when the channel is
-		 * handed in.
-		 *
-		 * @param kvStateId                 ID of the KvState instance to query
-		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
-		 *                                  with
-		 * @return Future holding the serialized result
-		 */
-		public Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-			synchronized (connectLock) {
-				if (failureCause != null) {
-					return Futures.failed(failureCause);
-				} else if (closed) {
-					return Futures.failed(new ClosedChannelException());
-				} else {
-					if (established != null) {
-						return established.getKvState(kvStateId, serializedKeyAndNamespace);
-					} else {
-						// Queue this and handle when connected
-						PendingRequest pending = new PendingRequest(kvStateId, serializedKeyAndNamespace);
-						queuedRequests.add(pending);
-						return pending.promise.future();
-					}
-				}
-			}
-		}
-
-		/**
-		 * Hands in a channel after a successful connection.
-		 *
-		 * @param channel Channel to hand in
-		 */
-		private void handInChannel(Channel channel) {
-			synchronized (connectLock) {
-				if (closed || failureCause != null) {
-					// Close the channel and we are done. Any queued requests
-					// are removed on the close/failure call and after that no
-					// new ones can be enqueued.
-					channel.close();
-				} else {
-					established = new EstablishedConnection(serverAddress, channel);
-
-					PendingRequest pending;
-					while ((pending = queuedRequests.poll()) != null) {
-						Future<byte[]> resultFuture = established.getKvState(
-								pending.kvStateId,
-								pending.serializedKeyAndNamespace);
-
-						pending.promise.completeWith(resultFuture);
-					}
-
-					// Publish the channel for the general public
-					establishedConnections.put(serverAddress, established);
-					pendingConnections.remove(serverAddress);
-
-					// Check shut down for possible race with shut down. We
-					// don't want any lingering connections after shut down,
-					// which can happen if we don't check this here.
-					if (shutDown.get()) {
-						if (establishedConnections.remove(serverAddress, established)) {
-							established.close();
-						}
-					}
-				}
-			}
-		}
-
-		/**
-		 * Close the connecting channel with a ClosedChannelException.
-		 */
-		private void close() {
-			close(new ClosedChannelException());
-		}
-
-		/**
-		 * Close the connecting channel with an Exception (can be
-		 * <code>null</code>) or forward to the established channel.
-		 */
-		private void close(Throwable cause) {
-			synchronized (connectLock) {
-				if (!closed) {
-					if (failureCause == null) {
-						failureCause = cause;
-					}
-
-					if (established != null) {
-						established.close();
-					} else {
-						PendingRequest pending;
-						while ((pending = queuedRequests.poll()) != null) {
-							pending.promise.tryFailure(cause);
-						}
-					}
-
-					closed = true;
-				}
-			}
-		}
-
-		/**
-		 * A pending request queued while the channel is connecting.
-		 */
-		private final class PendingRequest {
-
-			private final KvStateID kvStateId;
-			private final byte[] serializedKeyAndNamespace;
-			private final Promise<byte[]> promise;
-
-			private PendingRequest(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-				this.kvStateId = kvStateId;
-				this.serializedKeyAndNamespace = serializedKeyAndNamespace;
-				this.promise = Futures.promise();
-			}
-		}
-
-		@Override
-		public String toString() {
-			synchronized (connectLock) {
-				return "PendingConnection{" +
-						"serverAddress=" + serverAddress +
-						", queuedRequests=" + queuedRequests.size() +
-						", established=" + (established != null) +
-						", closed=" + closed +
-						'}';
-			}
-		}
-	}
-
-	/**
-	 * An established connection that wraps the actual channel instance and is
-	 * registered at the {@link KvStateClientHandler} for callbacks.
-	 */
-	private class EstablishedConnection implements KvStateClientHandlerCallback {
-
-		/** Address of the server we are connected to. */
-		private final KvStateServerAddress serverAddress;
-
-		/** The actual TCP channel. */
-		private final Channel channel;
-
-		/** Pending requests keyed by request ID. */
-		private final ConcurrentHashMap<Long, PromiseAndTimestamp> pendingRequests = new ConcurrentHashMap<>();
-
-		/** Current request number used to assign unique request IDs. */
-		private final AtomicLong requestCount = new AtomicLong();
-
-		/** Reference to a failure that was reported by the channel. */
-		private final AtomicReference<Throwable> failureCause = new AtomicReference<>();
-
-		/**
-		 * Creates an established connection with the given channel.
-		 *
-		 * @param serverAddress Address of the server connected to
-		 * @param channel The actual TCP channel
-		 */
-		EstablishedConnection(KvStateServerAddress serverAddress, Channel channel) {
-			this.serverAddress = Preconditions.checkNotNull(serverAddress, "KvStateServerAddress");
-			this.channel = Preconditions.checkNotNull(channel, "Channel");
-
-			// Add the client handler with the callback
-			channel.pipeline().addLast("KvStateClientHandler", new KvStateClientHandler(this));
-
-			stats.reportActiveConnection();
-		}
-
-		/**
-		 * Close the channel with a ClosedChannelException.
-		 */
-		void close() {
-			close(new ClosedChannelException());
-		}
-
-		/**
-		 * Close the channel with a cause.
-		 *
-		 * @param cause The cause to close the channel with.
-		 * @return Channel close future
-		 */
-		private boolean close(Throwable cause) {
-			if (failureCause.compareAndSet(null, cause)) {
-				channel.close();
-				stats.reportInactiveConnection();
-
-				for (long requestId : pendingRequests.keySet()) {
-					PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-					if (pending != null && pending.promise.tryFailure(cause)) {
-						stats.reportFailedRequest();
-					}
-				}
-
-				return true;
-			}
-
-			return false;
-		}
-
-		/**
-		 * Returns a future holding the serialized request result.
-		 *
-		 * @param kvStateId                 ID of the KvState instance to query
-		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
-		 *                                  with
-		 * @return Future holding the serialized result
-		 */
-		Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-			PromiseAndTimestamp requestPromiseTs = new PromiseAndTimestamp(
-					Futures.<byte[]>promise(),
-					System.nanoTime());
-
-			try {
-				final long requestId = requestCount.getAndIncrement();
-				pendingRequests.put(requestId, requestPromiseTs);
-
-				stats.reportRequest();
-
-				ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequest(
-						channel.alloc(),
-						requestId,
-						kvStateId,
-						serializedKeyAndNamespace);
-
-				channel.writeAndFlush(buf).addListener(new ChannelFutureListener() {
-					@Override
-					public void operationComplete(ChannelFuture future) throws Exception {
-						if (!future.isSuccess()) {
-							// Fail promise if not failed to write
-							PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-							if (pending != null && pending.promise.tryFailure(future.cause())) {
-								stats.reportFailedRequest();
-							}
-						}
-					}
-				});
-
-				// Check failure for possible race. We don't want any lingering
-				// promises after a failure, which can happen if we don't check
-				// this here. Note that close is treated as a failure as well.
-				Throwable failure = failureCause.get();
-				if (failure != null) {
-					// Remove from pending requests to guard against concurrent
-					// removal and to make sure that we only count it once as failed.
-					PromiseAndTimestamp p = pendingRequests.remove(requestId);
-					if (p != null && p.promise.tryFailure(failure)) {
-						stats.reportFailedRequest();
-					}
-				}
-			} catch (Throwable t) {
-				requestPromiseTs.promise.tryFailure(t);
-			}
-
-			return requestPromiseTs.promise.future();
-		}
-
-		@Override
-		public void onRequestResult(long requestId, byte[] serializedValue) {
-			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-			if (pending != null && pending.promise.trySuccess(serializedValue)) {
-				long durationMillis = (System.nanoTime() - pending.timestamp) / 1_000_000;
-				stats.reportSuccessfulRequest(durationMillis);
-			}
-		}
-
-		@Override
-		public void onRequestFailure(long requestId, Throwable cause) {
-			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-			if (pending != null && pending.promise.tryFailure(cause)) {
-				stats.reportFailedRequest();
-			}
-		}
-
-		@Override
-		public void onFailure(Throwable cause) {
-			if (close(cause)) {
-				// Remove from established channels, otherwise future
-				// requests will be handled by this failed channel.
-				establishedConnections.remove(serverAddress, this);
-			}
-		}
-
-		@Override
-		public String toString() {
-			return "EstablishedConnection{" +
-					"serverAddress=" + serverAddress +
-					", channel=" + channel +
-					", pendingRequests=" + pendingRequests.size() +
-					", requestCount=" + requestCount +
-					", failureCause=" + failureCause +
-					'}';
-		}
-
-		/**
-		 * Pair of promise and a timestamp.
-		 */
-		private class PromiseAndTimestamp {
-
-			private final Promise<byte[]> promise;
-			private final long timestamp;
-
-			public PromiseAndTimestamp(Promise<byte[]> promise, long timestamp) {
-				this.promise = promise;
-				this.timestamp = timestamp;
-			}
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
deleted file mode 100644
index 3e6470b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandler.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.query.netty;
-
-import org.apache.flink.runtime.query.netty.message.KvStateRequestFailure;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestResult;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestType;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.channels.ClosedChannelException;
-
-/**
- * This handler expects responses from {@link KvStateServerHandler}.
- *
- * <p>It deserializes the response and calls the registered callback, which is
- * responsible for actually handling the result (see {@link KvStateClient.EstablishedConnection}).
- */
-class KvStateClientHandler extends ChannelInboundHandlerAdapter {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientHandler.class);
-
-	private final KvStateClientHandlerCallback callback;
-
-	/**
-	 * Creates a {@link KvStateClientHandler} with the callback.
-	 *
-	 * @param callback Callback for responses.
-	 */
-	KvStateClientHandler(KvStateClientHandlerCallback callback) {
-		this.callback = callback;
-	}
-
-	@Override
-	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-		try {
-			ByteBuf buf = (ByteBuf) msg;
-			KvStateRequestType msgType = KvStateRequestSerializer.deserializeHeader(buf);
-
-			if (msgType == KvStateRequestType.REQUEST_RESULT) {
-				KvStateRequestResult result = KvStateRequestSerializer.deserializeKvStateRequestResult(buf);
-				callback.onRequestResult(result.getRequestId(), result.getSerializedResult());
-			} else if (msgType == KvStateRequestType.REQUEST_FAILURE) {
-				KvStateRequestFailure failure = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-				callback.onRequestFailure(failure.getRequestId(), failure.getCause());
-			} else if (msgType == KvStateRequestType.SERVER_FAILURE) {
-				throw KvStateRequestSerializer.deserializeServerFailure(buf);
-			} else {
-				throw new IllegalStateException("Unexpected response type '" + msgType + "'");
-			}
-		} catch (Throwable t1) {
-			try {
-				callback.onFailure(t1);
-			} catch (Throwable t2) {
-				LOG.error("Failed to notify callback about failure", t2);
-			}
-		} finally {
-			ReferenceCountUtil.release(msg);
-		}
-	}
-
-	@Override
-	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-		try {
-			callback.onFailure(cause);
-		} catch (Throwable t) {
-			LOG.error("Failed to notify callback about failure", t);
-		}
-	}
-
-	@Override
-	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-		// Only the client is expected to close the channel. Otherwise it
-		// indicates a failure. Note that this will be invoked in both cases
-		// though. If the callback closed the channel, the callback must be
-		// ignored.
-		try {
-			callback.onFailure(new ClosedChannelException());
-		} catch (Throwable t) {
-			LOG.error("Failed to notify callback about failure", t);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerCallback.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerCallback.java
deleted file mode 100644
index 65ff781..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerCallback.java
+++ /dev/null
@@ -1,54 +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.query.netty;
-
-import org.apache.flink.runtime.query.netty.message.KvStateRequest;
-
-/**
- * Callback for {@link KvStateClientHandler}.
- */
-interface KvStateClientHandlerCallback {
-
-	/**
-	 * Called on a successful {@link KvStateRequest}.
-	 *
-	 * @param requestId       ID of the request
-	 * @param serializedValue Serialized value for the request
-	 */
-	void onRequestResult(long requestId, byte[] serializedValue);
-
-	/**
-	 * Called on a failed {@link KvStateRequest}.
-	 *
-	 * @param requestId ID of the request
-	 * @param cause     Cause of the request failure
-	 */
-	void onRequestFailure(long requestId, Throwable cause);
-
-	/**
-	 * Called on any failure, which is not related to a specific request.
-	 *
-	 * <p>This can be for example a caught Exception in the channel pipeline
-	 * or an unexpected channel close.
-	 *
-	 * @param cause Cause of the failure
-	 */
-	void onFailure(Throwable cause);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
index 1c0d8d5..9781e23 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.query.netty;
 
+import org.apache.flink.runtime.query.KvStateServer;
+
 /**
  * Simple statistics for {@link KvStateServer} monitoring.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
deleted file mode 100644
index 7cf2148..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java
+++ /dev/null
@@ -1,239 +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.query.netty;
-
-import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.netty.message.KvStateRequest;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
-import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Netty-based server answering {@link KvStateRequest} messages.
- *
- * <p>Requests are handled by asynchronous query tasks (see {@link KvStateServerHandler.AsyncKvStateQueryTask})
- * that are executed by a separate query Thread pool. This pool is shared among
- * all TCP connections.
- *
- * <p>The incoming pipeline looks as follows:
- * <pre>
- * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
- * </pre>
- *
- * <p>Received binary messages are expected to contain a frame length field. Netty's
- * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
- * giving it to our {@link KvStateServerHandler}.
- *
- * <p>Connections are established and closed by the client. The server only
- * closes the connection on a fatal failure that cannot be recovered. A
- * server-side connection close is considered a failure by the client.
- */
-public class KvStateServer {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateServer.class);
-
-	/** Server config: low water mark. */
-	private static final int LOW_WATER_MARK = 8 * 1024;
-
-	/** Server config: high water mark. */
-	private static final int HIGH_WATER_MARK = 32 * 1024;
-
-	/** Netty's ServerBootstrap. */
-	private final ServerBootstrap bootstrap;
-
-	/** Query executor thread pool. */
-	private final ExecutorService queryExecutor;
-
-	/** Address of this server. */
-	private KvStateServerAddress serverAddress;
-
-	/**
-	 * Creates the {@link KvStateServer}.
-	 *
-	 * <p>The server needs to be started via {@link #start()} in order to bind
-	 * to the configured bind address.
-	 *
-	 * @param bindAddress         Address to bind to
-	 * @param bindPort            Port to bind to. Pick random port if 0.
-	 * @param numEventLoopThreads Number of event loop threads
-	 * @param numQueryThreads     Number of query threads
-	 * @param kvStateRegistry     KvStateRegistry to query for KvState instances
-	 * @param stats               Statistics tracker
-	 */
-	public KvStateServer(
-			InetAddress bindAddress,
-			int bindPort,
-			int numEventLoopThreads,
-			int numQueryThreads,
-			KvStateRegistry kvStateRegistry,
-			KvStateRequestStats stats) {
-
-		Preconditions.checkArgument(bindPort >= 0 && bindPort <= 65536, "Port " + bindPort +
-				" is out of valid port range (0-65536).");
-
-		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
-		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
-
-		Preconditions.checkNotNull(kvStateRegistry, "KvStateRegistry");
-		Preconditions.checkNotNull(stats, "KvStateRequestStats");
-
-		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
-
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateServer EventLoop Thread %d")
-				.build();
-
-		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
-
-		queryExecutor = createQueryExecutor(numQueryThreads);
-
-		// Shared between all channels
-		KvStateServerHandler serverHandler = new KvStateServerHandler(
-				kvStateRegistry,
-				queryExecutor,
-				stats);
-
-		bootstrap = new ServerBootstrap()
-				// Bind address and port
-				.localAddress(bindAddress, bindPort)
-				// NIO server channels
-				.group(nioGroup)
-				.channel(NioServerSocketChannel.class)
-				// Server channel Options
-				.option(ChannelOption.ALLOCATOR, bufferPool)
-				// Child channel options
-				.childOption(ChannelOption.ALLOCATOR, bufferPool)
-				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
-				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK)
-				// See initializer for pipeline details
-				.childHandler(new KvStateServerChannelInitializer(serverHandler));
-	}
-
-	/**
-	 * Starts the server by binding to the configured bind address (blocking).
-	 *
-	 * @throws InterruptedException If interrupted during the bind operation
-	 */
-	public void start() throws InterruptedException {
-		Channel channel = bootstrap.bind().sync().channel();
-
-		InetSocketAddress localAddress = (InetSocketAddress) channel.localAddress();
-		serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
-	}
-
-	/**
-	 * Returns the address of this server.
-	 *
-	 * @return Server address
-	 * @throws IllegalStateException If server has not been started yet
-	 */
-	public KvStateServerAddress getAddress() {
-		if (serverAddress == null) {
-			throw new IllegalStateException("KvStateServer not started yet.");
-		}
-
-		return serverAddress;
-	}
-
-	/**
-	 * Shuts down the server and all related thread pools.
-	 */
-	public void shutDown() {
-		if (bootstrap != null) {
-			EventLoopGroup group = bootstrap.group();
-			if (group != null) {
-				group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
-			}
-		}
-
-		if (queryExecutor != null) {
-			queryExecutor.shutdown();
-		}
-
-		serverAddress = null;
-	}
-
-	/**
-	 * Creates a thread pool for the query execution.
-	 *
-	 * @param numQueryThreads Number of query threads.
-	 * @return Thread pool for query execution
-	 */
-	private static ExecutorService createQueryExecutor(int numQueryThreads) {
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateServer Query Thread %d")
-				.build();
-
-		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
-	}
-
-	/**
-	 * Channel pipeline initializer.
-	 *
-	 * <p>The request handler is shared, whereas the other handlers are created
-	 * per channel.
-	 */
-	private static final class KvStateServerChannelInitializer extends ChannelInitializer<SocketChannel> {
-
-		/** The shared request handler. */
-		private final KvStateServerHandler sharedRequestHandler;
-
-		/**
-		 * Creates the channel pipeline initializer with the shared request handler.
-		 *
-		 * @param sharedRequestHandler Shared request handler.
-		 */
-		public KvStateServerChannelInitializer(KvStateServerHandler sharedRequestHandler) {
-			this.sharedRequestHandler = Preconditions.checkNotNull(sharedRequestHandler, "Request handler");
-		}
-
-		@Override
-		protected void initChannel(SocketChannel ch) throws Exception {
-			ch.pipeline()
-					.addLast(new ChunkedWriteHandler())
-					.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-					.addLast(sharedRequestHandler);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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
deleted file mode 100644
index 1af55dc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServerHandler.java
+++ /dev/null
@@ -1,305 +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.query.netty;
-
-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.internal.InternalKvState;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Objects;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * 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
- * query task. The actual query is handled in a separate thread as it might
- * otherwise block the network threads (file I/O etc.).
- */
-@ChannelHandler.Sharable
-class KvStateServerHandler extends ChannelInboundHandlerAdapter {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerHandler.class);
-
-	/** KvState registry holding references to the KvState instances. */
-	private final KvStateRegistry registry;
-
-	/** Thread pool for query execution. */
-	private final ExecutorService queryExecutor;
-
-	/** Exposed server statistics. */
-	private final KvStateRequestStats stats;
-
-	/**
-	 * Create the handler.
-	 *
-	 * @param kvStateRegistry Registry to query.
-	 * @param queryExecutor   Thread pool for query execution.
-	 * @param stats           Exposed server statistics.
-	 */
-	KvStateServerHandler(
-			KvStateRegistry kvStateRegistry,
-			ExecutorService queryExecutor,
-			KvStateRequestStats stats) {
-
-		this.registry = Objects.requireNonNull(kvStateRegistry, "KvStateRegistry");
-		this.queryExecutor = Objects.requireNonNull(queryExecutor, "Query thread pool");
-		this.stats = Objects.requireNonNull(stats, "KvStateRequestStats");
-	}
-
-	@Override
-	public void channelActive(ChannelHandlerContext ctx) throws Exception {
-		stats.reportActiveConnection();
-	}
-
-	@Override
-	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-		stats.reportInactiveConnection();
-	}
-
-	@Override
-	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-		KvStateRequest request = null;
-
-		try {
-			ByteBuf buf = (ByteBuf) msg;
-			KvStateRequestType msgType = KvStateRequestSerializer.deserializeHeader(buf);
-
-			if (msgType == KvStateRequestType.REQUEST) {
-				// ------------------------------------------------------------
-				// Request
-				// ------------------------------------------------------------
-				request = KvStateRequestSerializer.deserializeKvStateRequest(buf);
-
-				stats.reportRequest();
-
-				InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
-
-				if (kvState != null) {
-					// Execute actual query async, because it is possibly
-					// blocking (e.g. file I/O).
-					//
-					// A submission failure is not treated as fatal.
-					queryExecutor.submit(new AsyncKvStateQueryTask(ctx, request, kvState, stats));
-				} else {
-					ByteBuf unknown = KvStateRequestSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(),
-							request.getRequestId(),
-							new UnknownKvStateID(request.getKvStateId()));
-
-					ctx.writeAndFlush(unknown);
-
-					stats.reportFailedRequest();
-				}
-			} else {
-				// ------------------------------------------------------------
-				// Unexpected
-				// ------------------------------------------------------------
-				ByteBuf failure = KvStateRequestSerializer.serializeServerFailure(
-						ctx.alloc(),
-						new IllegalArgumentException("Unexpected message type " + msgType
-								+ ". KvStateServerHandler expects "
-								+ KvStateRequestType.REQUEST + " messages."));
-
-				ctx.writeAndFlush(failure);
-			}
-		} catch (Throwable t) {
-			String stringifiedCause = ExceptionUtils.stringifyException(t);
-
-			ByteBuf err;
-			if (request != null) {
-				String errMsg = "Failed to handle incoming request with ID " +
-						request.getRequestId() + ". Caused by: " + stringifiedCause;
-				err = KvStateRequestSerializer.serializeKvStateRequestFailure(
-						ctx.alloc(),
-						request.getRequestId(),
-						new RuntimeException(errMsg));
-
-				stats.reportFailedRequest();
-			} else {
-				String errMsg = "Failed to handle incoming message. Caused by: " + stringifiedCause;
-				err = KvStateRequestSerializer.serializeServerFailure(
-						ctx.alloc(),
-						new RuntimeException(errMsg));
-			}
-
-			ctx.writeAndFlush(err);
-		} finally {
-			// IMPORTANT: We have to always recycle the incoming buffer.
-			// Otherwise we will leak memory out of Netty's buffer pool.
-			//
-			// If any operation ever holds on to the buffer, it is the
-			// responsibility of that operation to retain the buffer and
-			// release it later.
-			ReferenceCountUtil.release(msg);
-		}
-	}
-
-	@Override
-	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-		String stringifiedCause = ExceptionUtils.stringifyException(cause);
-		String msg = "Exception in server pipeline. Caused by: " + stringifiedCause;
-
-		ByteBuf err = KvStateRequestSerializer.serializeServerFailure(
-				ctx.alloc(),
-				new RuntimeException(msg));
-
-		ctx.writeAndFlush(err).addListener(ChannelFutureListener.CLOSE);
-	}
-
-	/**
-	 * Task to execute the actual query against the {@link InternalKvState} instance.
-	 */
-	private static class AsyncKvStateQueryTask implements Runnable {
-
-		private final ChannelHandlerContext ctx;
-
-		private final KvStateRequest request;
-
-		private final InternalKvState<?> kvState;
-
-		private final KvStateRequestStats stats;
-
-		private final long creationNanos;
-
-		public AsyncKvStateQueryTask(
-				ChannelHandlerContext ctx,
-				KvStateRequest request,
-				InternalKvState<?> kvState,
-				KvStateRequestStats stats) {
-
-			this.ctx = Objects.requireNonNull(ctx, "Channel handler context");
-			this.request = Objects.requireNonNull(request, "State query");
-			this.kvState = Objects.requireNonNull(kvState, "KvState");
-			this.stats = Objects.requireNonNull(stats, "State query stats");
-			this.creationNanos = System.nanoTime();
-		}
-
-		@Override
-		public void run() {
-			boolean success = false;
-
-			try {
-				if (!ctx.channel().isActive()) {
-					return;
-				}
-
-				// Query the KvState instance
-				byte[] serializedKeyAndNamespace = request.getSerializedKeyAndNamespace();
-				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
-
-				if (serializedResult != null) {
-					// We found some data, success!
-					ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequestResult(
-							ctx.alloc(),
-							request.getRequestId(),
-							serializedResult);
-
-					int highWatermark = ctx.channel().config().getWriteBufferHighWaterMark();
-
-					ChannelFuture write;
-					if (buf.readableBytes() <= highWatermark) {
-						write = ctx.writeAndFlush(buf);
-					} else {
-						write = ctx.writeAndFlush(new ChunkedByteBuf(buf, highWatermark));
-					}
-
-					write.addListener(new QueryResultWriteListener());
-
-					success = true;
-				} else {
-					// No data for the key/namespace. This is considered to be
-					// a failure.
-					ByteBuf unknownKey = KvStateRequestSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(),
-							request.getRequestId(),
-							new UnknownKeyOrNamespace());
-
-					ctx.writeAndFlush(unknownKey);
-				}
-			} catch (Throwable t) {
-				try {
-					String stringifiedCause = ExceptionUtils.stringifyException(t);
-					String errMsg = "Failed to query state backend for query " +
-							request.getRequestId() + ". Caused by: " + stringifiedCause;
-
-					ByteBuf err = KvStateRequestSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(), request.getRequestId(), new RuntimeException(errMsg));
-
-					ctx.writeAndFlush(err);
-				} catch (IOException e) {
-					LOG.error("Failed to respond with the error after failed to query state backend", e);
-				}
-			} finally {
-				if (!success) {
-					stats.reportFailedRequest();
-				}
-			}
-		}
-
-		@Override
-		public String toString() {
-			return "AsyncKvStateQueryTask{" +
-					", request=" + request +
-					", creationNanos=" + creationNanos +
-					'}';
-		}
-
-		/**
-		 * Callback after query result has been written.
-		 *
-		 * <p>Gathers stats and logs errors.
-		 */
-		private class QueryResultWriteListener implements ChannelFutureListener {
-
-			@Override
-			public void operationComplete(ChannelFuture future) throws Exception {
-				long durationNanos = System.nanoTime() - creationNanos;
-				long durationMillis = TimeUnit.MILLISECONDS.convert(durationNanos, TimeUnit.NANOSECONDS);
-
-				if (future.isSuccess()) {
-					stats.reportSuccessfulRequest(durationMillis);
-				} else {
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Query " + request + " failed after " + durationMillis + " ms", future.cause());
-					}
-
-					stats.reportFailedRequest();
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKeyOrNamespace.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKeyOrNamespace.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKeyOrNamespace.java
deleted file mode 100644
index 4e5a1de..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKeyOrNamespace.java
+++ /dev/null
@@ -1,31 +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.query.netty;
-
-/**
- * Thrown if the KvState does not hold any state for the given key or namespace.
- */
-public class UnknownKeyOrNamespace extends IllegalStateException {
-
-	private static final long serialVersionUID = 1L;
-
-	UnknownKeyOrNamespace() {
-		super("KvState does not hold any state for key/namespace.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKvStateID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKvStateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKvStateID.java
deleted file mode 100644
index cc60035..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/UnknownKvStateID.java
+++ /dev/null
@@ -1,35 +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.query.netty;
-
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.util.Preconditions;
-
-/**
- * Thrown if no KvState with the given ID cannot found by the server handler.
- */
-public class UnknownKvStateID extends IllegalStateException {
-
-	private static final long serialVersionUID = 1L;
-
-	public UnknownKvStateID(KvStateID kvStateId) {
-		super("No KvState registered with ID " + Preconditions.checkNotNull(kvStateId, "KvStateID") +
-				" at TaskManager.");
-	}
-}


[14/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
[FLINK-7770][QS] Hide the queryable state behind a proxy.

Previously the QueryableStateClient could connect to the JM
and the TMs directly to fetch the required state. Now, there
is a proxy running on each TM and the remote client connects
to one of these proxies in order to get its state. The proxy
receives the request from the client, performs all necessary
message exchanges within the Flink cluster, receives the state
and forwards it back to the client.

This architecture allows for more security features to be
integrated in the future, as the proxy is running in the
cluster, it exposes less information about the cluster to
the outside world, and is more suitable for containerized
environments.


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

Branch: refs/heads/master
Commit: f48f5340a871ac58a649766e434218b67e9322ac
Parents: 29a6e99
Author: kkloudas <kk...@gmail.com>
Authored: Thu Oct 5 15:16:23 2017 +0200
Committer: kkloudas <kk...@gmail.com>
Committed: Wed Oct 11 15:33:32 2017 +0200

----------------------------------------------------------------------
 .../configuration/QueryableStateOptions.java    |   2 +-
 .../java/org/apache/flink/util/AbstractID.java  |   4 +-
 .../flink/queryablestate/UnknownJobManager.java |  35 -
 .../UnknownJobManagerException.java             |  36 +
 .../queryablestate/UnknownKeyOrNamespace.java   |  31 -
 .../UnknownKeyOrNamespaceException.java         |  39 +
 .../flink/queryablestate/UnknownKvStateID.java  |  35 -
 .../UnknownKvStateIdException.java              |  42 +
 .../UnknownKvStateKeyGroupLocation.java         |  31 -
 ...UnknownKvStateKeyGroupLocationException.java |  41 +
 .../AkkaKvStateLocationLookupService.java       | 325 --------
 .../queryablestate/client/KvStateClient.java    | 583 --------------
 .../client/KvStateClientHandler.java            | 107 ---
 .../client/KvStateClientHandlerCallback.java    |  54 --
 .../client/KvStateLocationLookupService.java    |  51 --
 .../client/QueryableStateClient.java            | 479 +++--------
 .../client/proxy/KvStateClientProxyHandler.java | 225 ++++++
 .../client/proxy/KvStateClientProxyImpl.java    | 127 +++
 .../messages/KvStateInternalRequest.java        |  93 +++
 .../queryablestate/messages/KvStateRequest.java | 142 ++--
 .../messages/KvStateRequestFailure.java         |  68 --
 .../messages/KvStateRequestResult.java          |  74 --
 .../messages/KvStateResponse.java               |  75 ++
 .../network/AbstractServerBase.java             | 241 ++++++
 .../network/AbstractServerHandler.java          | 306 ++++++++
 .../network/BadRequestException.java            |  35 +
 .../queryablestate/network/ChunkedByteBuf.java  | 100 +++
 .../flink/queryablestate/network/Client.java    | 537 +++++++++++++
 .../queryablestate/network/ClientHandler.java   | 122 +++
 .../network/ClientHandlerCallback.java          |  56 ++
 .../network/messages/MessageBody.java           |  38 +
 .../network/messages/MessageDeserializer.java   |  39 +
 .../network/messages/MessageSerializer.java     | 228 +++---
 .../network/messages/MessageType.java           |   7 +-
 .../network/messages/RequestFailure.java        |  71 ++
 .../queryablestate/server/ChunkedByteBuf.java   |  98 ---
 .../server/KvStateServerHandler.java            | 279 +------
 .../server/KvStateServerImpl.java               | 222 ++----
 .../itcases/AbstractQueryableStateITCase.java   | 534 +++++++------
 .../itcases/HAAbstractQueryableStateITCase.java |   8 +-
 .../NonHAAbstractQueryableStateITCase.java      |  10 +-
 .../AkkaKvStateLocationLookupServiceTest.java   | 399 ----------
 .../queryablestate/network/ClientTest.java      | 784 +++++++++++++++++++
 .../network/KvStateClientHandlerTest.java       |  24 +-
 .../network/KvStateClientTest.java              | 752 ------------------
 .../network/KvStateRequestSerializerTest.java   | 214 -----
 .../network/KvStateServerHandlerTest.java       | 286 ++++---
 .../network/KvStateServerTest.java              |  30 +-
 .../network/MessageSerializerTest.java          | 220 ++++++
 .../network/QueryableStateClientTest.java       | 458 -----------
 .../flink/runtime/concurrent/FutureUtils.java   |  16 +
 .../runtime/io/network/NetworkEnvironment.java  |  44 +-
 .../flink/runtime/query/KvStateClientProxy.java |  65 ++
 .../flink/runtime/query/KvStateLocation.java    |   8 +-
 .../flink/runtime/query/KvStateServer.java      |  19 +-
 .../runtime/query/QueryableStateUtils.java      |  54 +-
 .../query/netty/KvStateRequestStats.java        |   6 +-
 .../QueryableStateConfiguration.java            |   1 +
 .../taskexecutor/TaskManagerServices.java       |  16 +-
 .../TaskManagerServicesConfiguration.java       |   4 +-
 .../flink/runtime/taskmanager/TaskManager.scala |  28 +-
 .../io/network/NetworkEnvironmentTest.java      |   1 +
 ...askManagerComponentsStartupShutdownTest.java |   1 +
 63 files changed, 4317 insertions(+), 4743 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java
index 8b17cfb..df850e9 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/QueryableStateOptions.java
@@ -40,7 +40,7 @@ public class QueryableStateOptions {
 	/** Port to bind KvState server to (0 => pick random available port). */
 	public static final ConfigOption<Integer> SERVER_PORT =
 			key("query.server.port")
-			.defaultValue(0);
+			.defaultValue(9069);
 
 	/** Number of network (event loop) threads for the KvState server (0 => #slots). */
 	public static final ConfigOption<Integer> SERVER_NETWORK_THREADS =

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
index 397bb71..12d634d 100644
--- a/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractID.java
@@ -150,7 +150,7 @@ public class AbstractID implements Comparable<AbstractID>, java.io.Serializable
 				((int)  this.upperPart) ^
 				((int) (this.upperPart >>> 32));
 	}
-	
+
 	@Override
 	public String toString() {
 		if (this.toString == null) {
@@ -163,7 +163,7 @@ public class AbstractID implements Comparable<AbstractID>, java.io.Serializable
 
 		return this.toString;
 	}
-	
+
 	@Override
 	public int compareTo(AbstractID o) {
 		int diff1 = (this.upperPart < o.upperPart) ? -1 : ((this.upperPart == o.upperPart) ? 0 : 1);

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
deleted file mode 100644
index 93f2ba5..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
+++ /dev/null
@@ -1,35 +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.queryablestate;
-
-import org.apache.flink.queryablestate.client.KvStateLocationLookupService;
-
-/**
- * Exception to fail Future with if no JobManager is currently registered at
- * the {@link KvStateLocationLookupService}.
- */
-public class UnknownJobManager extends Exception {
-
-	private static final long serialVersionUID = 1L;
-
-	public UnknownJobManager() {
-		super("Unknown JobManager. Either the JobManager has not registered yet " +
-				"or has lost leadership.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java
new file mode 100644
index 0000000..fa2604b
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManagerException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.queryablestate;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * Exception to fail Future if the Task Manager on which the
+ * {@link org.apache.flink.runtime.query.KvStateClientProxy}
+ * is running on, does not know the active Job Manager.
+ */
+@Internal
+public class UnknownJobManagerException extends Exception {
+
+	private static final long serialVersionUID = 9092442511708951209L;
+
+	public UnknownJobManagerException() {
+		super("Unknown JobManager. Either the JobManager has not registered yet or has lost leadership.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java
deleted file mode 100644
index e921e40..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java
+++ /dev/null
@@ -1,31 +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.queryablestate;
-
-/**
- * Thrown if the KvState does not hold any state for the given key or namespace.
- */
-public class UnknownKeyOrNamespace extends IllegalStateException {
-
-	private static final long serialVersionUID = 1L;
-
-	public UnknownKeyOrNamespace() {
-		super("KvState does not hold any state for key/namespace.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java
new file mode 100644
index 0000000..c497a72
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespaceException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.BadRequestException;
+
+/**
+ * Thrown if the KvState does not hold any state for the given key or namespace.
+ */
+@Internal
+public class UnknownKeyOrNamespaceException extends BadRequestException {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates the exception.
+	 * @param serverName the name of the server that threw the exception.
+	 */
+	public UnknownKeyOrNamespaceException(String serverName) {
+		super(serverName, "No state for the specified key/namespace.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
deleted file mode 100644
index d5ff828..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
+++ /dev/null
@@ -1,35 +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.queryablestate;
-
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.util.Preconditions;
-
-/**
- * Thrown if no KvState with the given ID cannot found by the server handler.
- */
-public class UnknownKvStateID extends IllegalStateException {
-
-	private static final long serialVersionUID = 1L;
-
-	public UnknownKvStateID(KvStateID kvStateId) {
-		super("No KvState registered with ID " + Preconditions.checkNotNull(kvStateId, "KvStateID") +
-				" at TaskManager.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java
new file mode 100644
index 0000000..59ba081
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateIdException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.BadRequestException;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Thrown if no KvState with the given ID cannot found by the server handler.
+ */
+@Internal
+public class UnknownKvStateIdException extends BadRequestException {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates the exception.
+	 * @param serverName the name of the server that threw the exception.
+	 * @param kvStateId the state id for which no state was found.
+	 */
+	public UnknownKvStateIdException(String serverName, KvStateID kvStateId) {
+		super(serverName, "No registered state with ID " + Preconditions.checkNotNull(kvStateId) + '.');
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java
deleted file mode 100644
index fd25fae..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java
+++ /dev/null
@@ -1,31 +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.queryablestate;
-
-import org.apache.flink.runtime.query.KvStateLocation;
-
-/**
- * Exception thrown if there is no location information available for the given
- * key group in a {@link KvStateLocation} instance.
- */
-public class UnknownKvStateKeyGroupLocation extends Exception {
-
-	private static final long serialVersionUID = 1L;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java
new file mode 100644
index 0000000..0d6588a
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocationException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.BadRequestException;
+import org.apache.flink.runtime.query.KvStateLocation;
+
+/**
+ * Exception thrown if there is no location information available for the given
+ * key group in a {@link KvStateLocation} instance.
+ */
+@Internal
+public class UnknownKvStateKeyGroupLocationException extends BadRequestException {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates the exception.
+	 * @param serverName the name of the server that threw the exception.
+	 */
+	public UnknownKvStateKeyGroupLocationException(String serverName) {
+		super(serverName, "Unknown key-group location.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
deleted file mode 100644
index f42e008..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
+++ /dev/null
@@ -1,325 +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.queryablestate.client;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.queryablestate.UnknownJobManager;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.query.KvStateLocation;
-import org.apache.flink.runtime.query.KvStateMessage;
-import org.apache.flink.util.Preconditions;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-import akka.dispatch.Recover;
-import akka.pattern.Patterns;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.UUID;
-import java.util.concurrent.Callable;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-import scala.reflect.ClassTag$;
-
-/**
- * Akka-based {@link KvStateLocationLookupService} that retrieves the current
- * JobManager address and uses it for lookups.
- */
-public class AkkaKvStateLocationLookupService implements KvStateLocationLookupService, LeaderRetrievalListener {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateLocationLookupService.class);
-
-	/** Future returned when no JobManager is available. */
-	private static final Future<ActorGateway> UNKNOWN_JOB_MANAGER = Futures.failed(new UnknownJobManager());
-
-	/** Leader retrieval service to retrieve the current job manager. */
-	private final LeaderRetrievalService leaderRetrievalService;
-
-	/** The actor system used to resolve the JobManager address. */
-	private final ActorSystem actorSystem;
-
-	/** Timeout for JobManager ask-requests. */
-	private final FiniteDuration askTimeout;
-
-	/** Retry strategy factory on future failures. */
-	private final LookupRetryStrategyFactory retryStrategyFactory;
-
-	/** Current job manager future. */
-	private volatile Future<ActorGateway> jobManagerFuture = UNKNOWN_JOB_MANAGER;
-
-	/**
-	 * Creates the Akka-based {@link KvStateLocationLookupService}.
-	 *
-	 * @param leaderRetrievalService Leader retrieval service to use.
-	 * @param actorSystem            Actor system to use.
-	 * @param askTimeout             Timeout for JobManager ask-requests.
-	 * @param retryStrategyFactory   Retry strategy if no JobManager available.
-	 */
-	public AkkaKvStateLocationLookupService(
-			LeaderRetrievalService leaderRetrievalService,
-			ActorSystem actorSystem,
-			FiniteDuration askTimeout,
-			LookupRetryStrategyFactory retryStrategyFactory) {
-
-		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService, "Leader retrieval service");
-		this.actorSystem = Preconditions.checkNotNull(actorSystem, "Actor system");
-		this.askTimeout = Preconditions.checkNotNull(askTimeout, "Ask Timeout");
-		this.retryStrategyFactory = Preconditions.checkNotNull(retryStrategyFactory, "Retry strategy factory");
-	}
-
-	public void start() {
-		try {
-			leaderRetrievalService.start(this);
-		} catch (Exception e) {
-			LOG.error("Failed to start leader retrieval service", e);
-			throw new RuntimeException(e);
-		}
-	}
-
-	public void shutDown() {
-		try {
-			leaderRetrievalService.stop();
-		} catch (Exception e) {
-			LOG.error("Failed to stop leader retrieval service", e);
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public Future<KvStateLocation> getKvStateLookupInfo(final JobID jobId, final String registrationName) {
-		return getKvStateLookupInfo(jobId, registrationName, retryStrategyFactory.createRetryStrategy());
-	}
-
-	/**
-	 * Returns a future holding the {@link KvStateLocation} for the given job
-	 * and KvState registration name.
-	 *
-	 * <p>If there is currently no JobManager registered with the service, the
-	 * request is retried. The retry behaviour is specified by the
-	 * {@link LookupRetryStrategy} of the lookup service.
-	 *
-	 * @param jobId               JobID the KvState instance belongs to
-	 * @param registrationName    Name under which the KvState has been registered
-	 * @param lookupRetryStrategy Retry strategy to use for retries on UnknownJobManager failures.
-	 * @return Future holding the {@link KvStateLocation}
-	 */
-	@SuppressWarnings("unchecked")
-	private Future<KvStateLocation> getKvStateLookupInfo(
-			final JobID jobId,
-			final String registrationName,
-			final LookupRetryStrategy lookupRetryStrategy) {
-
-		return jobManagerFuture
-				.flatMap(new Mapper<ActorGateway, Future<Object>>() {
-					@Override
-					public Future<Object> apply(ActorGateway jobManager) {
-						// Lookup the KvStateLocation
-						Object msg = new KvStateMessage.LookupKvStateLocation(jobId, registrationName);
-						return jobManager.ask(msg, askTimeout);
-					}
-				}, actorSystem.dispatcher())
-				.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class))
-				.recoverWith(new Recover<Future<KvStateLocation>>() {
-					@Override
-					public Future<KvStateLocation> recover(Throwable failure) throws Throwable {
-						// If the Future fails with UnknownJobManager, retry
-						// the request. Otherwise all Futures will be failed
-						// during the start up phase, when the JobManager did
-						// not notify this service yet or leadership is lost
-						// intermittently.
-						if (failure instanceof UnknownJobManager && lookupRetryStrategy.tryRetry()) {
-							return Patterns.after(
-									lookupRetryStrategy.getRetryDelay(),
-									actorSystem.scheduler(),
-									actorSystem.dispatcher(),
-									new Callable<Future<KvStateLocation>>() {
-										@Override
-										public Future<KvStateLocation> call() throws Exception {
-											return getKvStateLookupInfo(
-													jobId,
-													registrationName,
-													lookupRetryStrategy);
-										}
-									});
-						} else {
-							return Futures.failed(failure);
-						}
-					}
-				}, actorSystem.dispatcher());
-	}
-
-	@Override
-	public void notifyLeaderAddress(String leaderAddress, final UUID leaderSessionID) {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Received leader address notification {}:{}", leaderAddress, leaderSessionID);
-		}
-
-		if (leaderAddress == null) {
-			jobManagerFuture = UNKNOWN_JOB_MANAGER;
-		} else {
-			jobManagerFuture = AkkaUtils.getActorRefFuture(leaderAddress, actorSystem, askTimeout)
-					.map(new Mapper<ActorRef, ActorGateway>() {
-						@Override
-						public ActorGateway apply(ActorRef actorRef) {
-							return new AkkaActorGateway(actorRef, leaderSessionID);
-						}
-					}, actorSystem.dispatcher());
-		}
-	}
-
-	@Override
-	public void handleError(Exception exception) {
-		jobManagerFuture = Futures.failed(exception);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Retry strategy for failed lookups.
-	 *
-	 * <p>Usage:
-	 * <pre>
-	 * LookupRetryStrategy retryStrategy = LookupRetryStrategyFactory.create();
-	 *
-	 * if (retryStrategy.tryRetry()) {
-	 *     // OK to retry
-	 *     FiniteDuration retryDelay = retryStrategy.getRetryDelay();
-	 * }
-	 * </pre>
-	 */
-	public interface LookupRetryStrategy {
-
-		/**
-		 * Returns the current retry.
-		 *
-		 * @return Current retry delay.
-		 */
-		FiniteDuration getRetryDelay();
-
-		/**
-		 * Tries another retry and returns whether it is allowed or not.
-		 *
-		 * @return Whether it is allowed to do another restart or not.
-		 */
-		boolean tryRetry();
-
-	}
-
-	/**
-	 * Factory for retry strategies.
-	 */
-	public interface LookupRetryStrategyFactory {
-
-		/**
-		 * Creates a new retry strategy.
-		 *
-		 * @return The retry strategy.
-		 */
-		LookupRetryStrategy createRetryStrategy();
-
-	}
-
-	/**
-	 * Factory for disabled retries.
-	 */
-	public static class DisabledLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
-
-		private static final DisabledLookupRetryStrategy RETRY_STRATEGY = new DisabledLookupRetryStrategy();
-
-		@Override
-		public LookupRetryStrategy createRetryStrategy() {
-			return RETRY_STRATEGY;
-		}
-
-		private static class DisabledLookupRetryStrategy implements LookupRetryStrategy {
-
-			@Override
-			public FiniteDuration getRetryDelay() {
-				return FiniteDuration.Zero();
-			}
-
-			@Override
-			public boolean tryRetry() {
-				return false;
-			}
-		}
-
-	}
-
-	/**
-	 * Factory for fixed delay retries.
-	 */
-	public static class FixedDelayLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
-
-		private final int maxRetries;
-		private final FiniteDuration retryDelay;
-
-		FixedDelayLookupRetryStrategyFactory(int maxRetries, FiniteDuration retryDelay) {
-			this.maxRetries = maxRetries;
-			this.retryDelay = retryDelay;
-		}
-
-		@Override
-		public LookupRetryStrategy createRetryStrategy() {
-			return new FixedDelayLookupRetryStrategy(maxRetries, retryDelay);
-		}
-
-		private static class FixedDelayLookupRetryStrategy implements LookupRetryStrategy {
-
-			private final Object retryLock = new Object();
-			private final int maxRetries;
-			private final FiniteDuration retryDelay;
-			private int numRetries;
-
-			public FixedDelayLookupRetryStrategy(int maxRetries, FiniteDuration retryDelay) {
-				Preconditions.checkArgument(maxRetries >= 0, "Negative number maximum retries");
-				this.maxRetries = maxRetries;
-				this.retryDelay = Preconditions.checkNotNull(retryDelay, "Retry delay");
-			}
-
-			@Override
-			public FiniteDuration getRetryDelay() {
-				synchronized (retryLock) {
-					return retryDelay;
-				}
-			}
-
-			@Override
-			public boolean tryRetry() {
-				synchronized (retryLock) {
-					if (numRetries < maxRetries) {
-						numRetries++;
-						return true;
-					} else {
-						return false;
-					}
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
deleted file mode 100644
index d456cd7..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
+++ /dev/null
@@ -1,583 +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.queryablestate.client;
-
-import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
-import org.apache.flink.queryablestate.UnknownKvStateID;
-import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateServer;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.netty.KvStateRequestStats;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
-import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
-import akka.dispatch.Futures;
-
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayDeque;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-
-/**
- * Netty-based client querying {@link KvStateServer} instances.
- *
- * <p>This client can be used by multiple threads concurrently. Operations are
- * executed asynchronously and return Futures to their result.
- *
- * <p>The incoming pipeline looks as follows:
- * <pre>
- * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
- * </pre>
- *
- * <p>Received binary messages are expected to contain a frame length field. Netty's
- * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
- * giving it to our {@link KvStateClientHandler}.
- *
- * <p>Connections are established and closed by the client. The server only
- * closes the connection on a fatal failure that cannot be recovered.
- */
-public class KvStateClient {
-
-	/** Netty's Bootstrap. */
-	private final Bootstrap bootstrap;
-
-	/** Statistics tracker. */
-	private final KvStateRequestStats stats;
-
-	/** Established connections. */
-	private final ConcurrentHashMap<KvStateServerAddress, EstablishedConnection> establishedConnections =
-			new ConcurrentHashMap<>();
-
-	/** Pending connections. */
-	private final ConcurrentHashMap<KvStateServerAddress, PendingConnection> pendingConnections =
-			new ConcurrentHashMap<>();
-
-	/** Atomic shut down flag. */
-	private final AtomicBoolean shutDown = new AtomicBoolean();
-
-	/**
-	 * Creates a client with the specified number of event loop threads.
-	 *
-	 * @param numEventLoopThreads Number of event loop threads (minimum 1).
-	 */
-	public KvStateClient(int numEventLoopThreads, KvStateRequestStats stats) {
-		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
-		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
-
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateClient Event Loop Thread %d")
-				.build();
-
-		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
-
-		this.bootstrap = new Bootstrap()
-				.group(nioGroup)
-				.channel(NioSocketChannel.class)
-				.option(ChannelOption.ALLOCATOR, bufferPool)
-				.handler(new ChannelInitializer<SocketChannel>() {
-					@Override
-					protected void initChannel(SocketChannel ch) throws Exception {
-						ch.pipeline()
-								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-								// ChunkedWriteHandler respects Channel writability
-								.addLast(new ChunkedWriteHandler());
-					}
-				});
-
-		this.stats = Preconditions.checkNotNull(stats, "Statistics tracker");
-	}
-
-	/**
-	 * Returns a future holding the serialized request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param serverAddress Address of the server to query
-	 * @param kvStateId ID of the KvState instance to query
-	 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance with
-	 * @return Future holding the serialized result
-	 */
-	public Future<byte[]> getKvState(
-			KvStateServerAddress serverAddress,
-			KvStateID kvStateId,
-			byte[] serializedKeyAndNamespace) {
-
-		if (shutDown.get()) {
-			return Futures.failed(new IllegalStateException("Shut down"));
-		}
-
-		EstablishedConnection connection = establishedConnections.get(serverAddress);
-
-		if (connection != null) {
-			return connection.getKvState(kvStateId, serializedKeyAndNamespace);
-		} else {
-			PendingConnection pendingConnection = pendingConnections.get(serverAddress);
-			if (pendingConnection != null) {
-				// There was a race, use the existing pending connection.
-				return pendingConnection.getKvState(kvStateId, serializedKeyAndNamespace);
-			} else {
-				// We try to connect to the server.
-				PendingConnection pending = new PendingConnection(serverAddress);
-				PendingConnection previous = pendingConnections.putIfAbsent(serverAddress, pending);
-
-				if (previous == null) {
-					// OK, we are responsible to connect.
-					bootstrap.connect(serverAddress.getHost(), serverAddress.getPort())
-							.addListener(pending);
-
-					return pending.getKvState(kvStateId, serializedKeyAndNamespace);
-				} else {
-					// There was a race, use the existing pending connection.
-					return previous.getKvState(kvStateId, serializedKeyAndNamespace);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Shuts down the client and closes all connections.
-	 *
-	 * <p>After a call to this method, all returned futures will be failed.
-	 */
-	public void shutDown() {
-		if (shutDown.compareAndSet(false, true)) {
-			for (Map.Entry<KvStateServerAddress, EstablishedConnection> conn : establishedConnections.entrySet()) {
-				if (establishedConnections.remove(conn.getKey(), conn.getValue())) {
-					conn.getValue().close();
-				}
-			}
-
-			for (Map.Entry<KvStateServerAddress, PendingConnection> conn : pendingConnections.entrySet()) {
-				if (pendingConnections.remove(conn.getKey()) != null) {
-					conn.getValue().close();
-				}
-			}
-
-			if (bootstrap != null) {
-				EventLoopGroup group = bootstrap.group();
-				if (group != null) {
-					group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Closes the connection to the given server address if it exists.
-	 *
-	 * <p>If there is a request to the server a new connection will be established.
-	 *
-	 * @param serverAddress Target address of the connection to close
-	 */
-	public void closeConnection(KvStateServerAddress serverAddress) {
-		PendingConnection pending = pendingConnections.get(serverAddress);
-		if (pending != null) {
-			pending.close();
-		}
-
-		EstablishedConnection established = establishedConnections.remove(serverAddress);
-		if (established != null) {
-			established.close();
-		}
-	}
-
-	/**
-	 * A pending connection that is in the process of connecting.
-	 */
-	private class PendingConnection implements ChannelFutureListener {
-
-		/** Lock to guard the connect call, channel hand in, etc. */
-		private final Object connectLock = new Object();
-
-		/** Address of the server we are connecting to. */
-		private final KvStateServerAddress serverAddress;
-
-		/** Queue of requests while connecting. */
-		private final ArrayDeque<PendingRequest> queuedRequests = new ArrayDeque<>();
-
-		/** The established connection after the connect succeeds. */
-		private EstablishedConnection established;
-
-		/** Closed flag. */
-		private boolean closed;
-
-		/** Failure cause if something goes wrong. */
-		private Throwable failureCause;
-
-		/**
-		 * Creates a pending connection to the given server.
-		 *
-		 * @param serverAddress Address of the server to connect to.
-		 */
-		private PendingConnection(KvStateServerAddress serverAddress) {
-			this.serverAddress = serverAddress;
-		}
-
-		@Override
-		public void operationComplete(ChannelFuture future) throws Exception {
-			// Callback from the Bootstrap's connect call.
-			if (future.isSuccess()) {
-				handInChannel(future.channel());
-			} else {
-				close(future.cause());
-			}
-		}
-
-		/**
-		 * Returns a future holding the serialized request result.
-		 *
-		 * <p>If the channel has been established, forward the call to the
-		 * established channel, otherwise queue it for when the channel is
-		 * handed in.
-		 *
-		 * @param kvStateId                 ID of the KvState instance to query
-		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
-		 *                                  with
-		 * @return Future holding the serialized result
-		 */
-		public Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-			synchronized (connectLock) {
-				if (failureCause != null) {
-					return Futures.failed(failureCause);
-				} else if (closed) {
-					return Futures.failed(new ClosedChannelException());
-				} else {
-					if (established != null) {
-						return established.getKvState(kvStateId, serializedKeyAndNamespace);
-					} else {
-						// Queue this and handle when connected
-						PendingRequest pending = new PendingRequest(kvStateId, serializedKeyAndNamespace);
-						queuedRequests.add(pending);
-						return pending.promise.future();
-					}
-				}
-			}
-		}
-
-		/**
-		 * Hands in a channel after a successful connection.
-		 *
-		 * @param channel Channel to hand in
-		 */
-		private void handInChannel(Channel channel) {
-			synchronized (connectLock) {
-				if (closed || failureCause != null) {
-					// Close the channel and we are done. Any queued requests
-					// are removed on the close/failure call and after that no
-					// new ones can be enqueued.
-					channel.close();
-				} else {
-					established = new EstablishedConnection(serverAddress, channel);
-
-					PendingRequest pending;
-					while ((pending = queuedRequests.poll()) != null) {
-						Future<byte[]> resultFuture = established.getKvState(
-								pending.kvStateId,
-								pending.serializedKeyAndNamespace);
-
-						pending.promise.completeWith(resultFuture);
-					}
-
-					// Publish the channel for the general public
-					establishedConnections.put(serverAddress, established);
-					pendingConnections.remove(serverAddress);
-
-					// Check shut down for possible race with shut down. We
-					// don't want any lingering connections after shut down,
-					// which can happen if we don't check this here.
-					if (shutDown.get()) {
-						if (establishedConnections.remove(serverAddress, established)) {
-							established.close();
-						}
-					}
-				}
-			}
-		}
-
-		/**
-		 * Close the connecting channel with a ClosedChannelException.
-		 */
-		private void close() {
-			close(new ClosedChannelException());
-		}
-
-		/**
-		 * Close the connecting channel with an Exception (can be
-		 * <code>null</code>) or forward to the established channel.
-		 */
-		private void close(Throwable cause) {
-			synchronized (connectLock) {
-				if (!closed) {
-					if (failureCause == null) {
-						failureCause = cause;
-					}
-
-					if (established != null) {
-						established.close();
-					} else {
-						PendingRequest pending;
-						while ((pending = queuedRequests.poll()) != null) {
-							pending.promise.tryFailure(cause);
-						}
-					}
-
-					closed = true;
-				}
-			}
-		}
-
-		/**
-		 * A pending request queued while the channel is connecting.
-		 */
-		private final class PendingRequest {
-
-			private final KvStateID kvStateId;
-			private final byte[] serializedKeyAndNamespace;
-			private final Promise<byte[]> promise;
-
-			private PendingRequest(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-				this.kvStateId = kvStateId;
-				this.serializedKeyAndNamespace = serializedKeyAndNamespace;
-				this.promise = Futures.promise();
-			}
-		}
-
-		@Override
-		public String toString() {
-			synchronized (connectLock) {
-				return "PendingConnection{" +
-						"serverAddress=" + serverAddress +
-						", queuedRequests=" + queuedRequests.size() +
-						", established=" + (established != null) +
-						", closed=" + closed +
-						'}';
-			}
-		}
-	}
-
-	/**
-	 * An established connection that wraps the actual channel instance and is
-	 * registered at the {@link KvStateClientHandler} for callbacks.
-	 */
-	private class EstablishedConnection implements KvStateClientHandlerCallback {
-
-		/** Address of the server we are connected to. */
-		private final KvStateServerAddress serverAddress;
-
-		/** The actual TCP channel. */
-		private final Channel channel;
-
-		/** Pending requests keyed by request ID. */
-		private final ConcurrentHashMap<Long, PromiseAndTimestamp> pendingRequests = new ConcurrentHashMap<>();
-
-		/** Current request number used to assign unique request IDs. */
-		private final AtomicLong requestCount = new AtomicLong();
-
-		/** Reference to a failure that was reported by the channel. */
-		private final AtomicReference<Throwable> failureCause = new AtomicReference<>();
-
-		/**
-		 * Creates an established connection with the given channel.
-		 *
-		 * @param serverAddress Address of the server connected to
-		 * @param channel The actual TCP channel
-		 */
-		EstablishedConnection(KvStateServerAddress serverAddress, Channel channel) {
-			this.serverAddress = Preconditions.checkNotNull(serverAddress, "KvStateServerAddress");
-			this.channel = Preconditions.checkNotNull(channel, "Channel");
-
-			// Add the client handler with the callback
-			channel.pipeline().addLast("KvStateClientHandler", new KvStateClientHandler(this));
-
-			stats.reportActiveConnection();
-		}
-
-		/**
-		 * Close the channel with a ClosedChannelException.
-		 */
-		void close() {
-			close(new ClosedChannelException());
-		}
-
-		/**
-		 * Close the channel with a cause.
-		 *
-		 * @param cause The cause to close the channel with.
-		 * @return Channel close future
-		 */
-		private boolean close(Throwable cause) {
-			if (failureCause.compareAndSet(null, cause)) {
-				channel.close();
-				stats.reportInactiveConnection();
-
-				for (long requestId : pendingRequests.keySet()) {
-					PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-					if (pending != null && pending.promise.tryFailure(cause)) {
-						stats.reportFailedRequest();
-					}
-				}
-
-				return true;
-			}
-
-			return false;
-		}
-
-		/**
-		 * Returns a future holding the serialized request result.
-		 *
-		 * @param kvStateId                 ID of the KvState instance to query
-		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
-		 *                                  with
-		 * @return Future holding the serialized result
-		 */
-		Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-			PromiseAndTimestamp requestPromiseTs = new PromiseAndTimestamp(
-					Futures.<byte[]>promise(),
-					System.nanoTime());
-
-			try {
-				final long requestId = requestCount.getAndIncrement();
-				pendingRequests.put(requestId, requestPromiseTs);
-
-				stats.reportRequest();
-
-				ByteBuf buf = MessageSerializer.serializeKvStateRequest(
-						channel.alloc(),
-						requestId,
-						kvStateId,
-						serializedKeyAndNamespace);
-
-				channel.writeAndFlush(buf).addListener(new ChannelFutureListener() {
-					@Override
-					public void operationComplete(ChannelFuture future) throws Exception {
-						if (!future.isSuccess()) {
-							// Fail promise if not failed to write
-							PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-							if (pending != null && pending.promise.tryFailure(future.cause())) {
-								stats.reportFailedRequest();
-							}
-						}
-					}
-				});
-
-				// Check failure for possible race. We don't want any lingering
-				// promises after a failure, which can happen if we don't check
-				// this here. Note that close is treated as a failure as well.
-				Throwable failure = failureCause.get();
-				if (failure != null) {
-					// Remove from pending requests to guard against concurrent
-					// removal and to make sure that we only count it once as failed.
-					PromiseAndTimestamp p = pendingRequests.remove(requestId);
-					if (p != null && p.promise.tryFailure(failure)) {
-						stats.reportFailedRequest();
-					}
-				}
-			} catch (Throwable t) {
-				requestPromiseTs.promise.tryFailure(t);
-			}
-
-			return requestPromiseTs.promise.future();
-		}
-
-		@Override
-		public void onRequestResult(long requestId, byte[] serializedValue) {
-			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-			if (pending != null && pending.promise.trySuccess(serializedValue)) {
-				long durationMillis = (System.nanoTime() - pending.timestamp) / 1_000_000;
-				stats.reportSuccessfulRequest(durationMillis);
-			}
-		}
-
-		@Override
-		public void onRequestFailure(long requestId, Throwable cause) {
-			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
-			if (pending != null && pending.promise.tryFailure(cause)) {
-				stats.reportFailedRequest();
-			}
-		}
-
-		@Override
-		public void onFailure(Throwable cause) {
-			if (close(cause)) {
-				// Remove from established channels, otherwise future
-				// requests will be handled by this failed channel.
-				establishedConnections.remove(serverAddress, this);
-			}
-		}
-
-		@Override
-		public String toString() {
-			return "EstablishedConnection{" +
-					"serverAddress=" + serverAddress +
-					", channel=" + channel +
-					", pendingRequests=" + pendingRequests.size() +
-					", requestCount=" + requestCount +
-					", failureCause=" + failureCause +
-					'}';
-		}
-
-		/**
-		 * Pair of promise and a timestamp.
-		 */
-		private class PromiseAndTimestamp {
-
-			private final Promise<byte[]> promise;
-			private final long timestamp;
-
-			public PromiseAndTimestamp(Promise<byte[]> promise, long timestamp) {
-				this.promise = promise;
-				this.timestamp = timestamp;
-			}
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
deleted file mode 100644
index 36a2b31..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
+++ /dev/null
@@ -1,107 +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.queryablestate.client;
-
-import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
-import org.apache.flink.queryablestate.messages.KvStateRequestResult;
-import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.queryablestate.network.messages.MessageType;
-import org.apache.flink.queryablestate.server.KvStateServerHandler;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.channels.ClosedChannelException;
-
-/**
- * This handler expects responses from {@link KvStateServerHandler}.
- *
- * <p>It deserializes the response and calls the registered callback, which is
- * responsible for actually handling the result (see {@link KvStateClient.EstablishedConnection}).
- */
-public class KvStateClientHandler extends ChannelInboundHandlerAdapter {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientHandler.class);
-
-	private final KvStateClientHandlerCallback callback;
-
-	/**
-	 * Creates a {@link KvStateClientHandler} with the callback.
-	 *
-	 * @param callback Callback for responses.
-	 */
-	public KvStateClientHandler(KvStateClientHandlerCallback callback) {
-		this.callback = callback;
-	}
-
-	@Override
-	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-		try {
-			ByteBuf buf = (ByteBuf) msg;
-			MessageType msgType = MessageSerializer.deserializeHeader(buf);
-
-			if (msgType == MessageType.REQUEST_RESULT) {
-				KvStateRequestResult result = MessageSerializer.deserializeKvStateRequestResult(buf);
-				callback.onRequestResult(result.getRequestId(), result.getSerializedResult());
-			} else if (msgType == MessageType.REQUEST_FAILURE) {
-				KvStateRequestFailure failure = MessageSerializer.deserializeKvStateRequestFailure(buf);
-				callback.onRequestFailure(failure.getRequestId(), failure.getCause());
-			} else if (msgType == MessageType.SERVER_FAILURE) {
-				throw MessageSerializer.deserializeServerFailure(buf);
-			} else {
-				throw new IllegalStateException("Unexpected response type '" + msgType + "'");
-			}
-		} catch (Throwable t1) {
-			try {
-				callback.onFailure(t1);
-			} catch (Throwable t2) {
-				LOG.error("Failed to notify callback about failure", t2);
-			}
-		} finally {
-			ReferenceCountUtil.release(msg);
-		}
-	}
-
-	@Override
-	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-		try {
-			callback.onFailure(cause);
-		} catch (Throwable t) {
-			LOG.error("Failed to notify callback about failure", t);
-		}
-	}
-
-	@Override
-	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-		// Only the client is expected to close the channel. Otherwise it
-		// indicates a failure. Note that this will be invoked in both cases
-		// though. If the callback closed the channel, the callback must be
-		// ignored.
-		try {
-			callback.onFailure(new ClosedChannelException());
-		} catch (Throwable t) {
-			LOG.error("Failed to notify callback about failure", t);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
deleted file mode 100644
index 98718fa..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
+++ /dev/null
@@ -1,54 +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.queryablestate.client;
-
-import org.apache.flink.queryablestate.messages.KvStateRequest;
-
-/**
- * Callback for {@link KvStateClientHandler}.
- */
-public interface KvStateClientHandlerCallback {
-
-	/**
-	 * Called on a successful {@link KvStateRequest}.
-	 *
-	 * @param requestId       ID of the request
-	 * @param serializedValue Serialized value for the request
-	 */
-	void onRequestResult(long requestId, byte[] serializedValue);
-
-	/**
-	 * Called on a failed {@link KvStateRequest}.
-	 *
-	 * @param requestId ID of the request
-	 * @param cause     Cause of the request failure
-	 */
-	void onRequestFailure(long requestId, Throwable cause);
-
-	/**
-	 * Called on any failure, which is not related to a specific request.
-	 *
-	 * <p>This can be for example a caught Exception in the channel pipeline
-	 * or an unexpected channel close.
-	 *
-	 * @param cause Cause of the failure
-	 */
-	void onFailure(Throwable cause);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
deleted file mode 100644
index 635cbae..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.queryablestate.client;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.query.KvStateLocation;
-
-import scala.concurrent.Future;
-
-/**
- * {@link KvStateLocation} lookup service.
- */
-public interface KvStateLocationLookupService {
-
-	/**
-	 * Starts the lookup service.
-	 */
-	void start();
-
-	/**
-	 * Shuts down the lookup service.
-	 */
-	void shutDown();
-
-	/**
-	 * Returns a future holding the {@link KvStateLocation} for the given job
-	 * and KvState registration name.
-	 *
-	 * @param jobId            JobID the KvState instance belongs to
-	 * @param registrationName Name under which the KvState has been registered
-	 * @return Future holding the {@link KvStateLocation}
-	 */
-	Future<KvStateLocation> getKvStateLookupInfo(JobID jobId, String registrationName);
-
-}


[13/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
index 27257d7..005c874 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
@@ -25,306 +25,117 @@ import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.ListSerializer;
-import org.apache.flink.configuration.AkkaOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.configuration.QueryableStateOptions;
-import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
-import org.apache.flink.queryablestate.UnknownKvStateID;
-import org.apache.flink.queryablestate.UnknownKvStateKeyGroupLocation;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.concurrent.Executors;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateLocation;
-import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.Client;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.runtime.concurrent.FutureUtils;
 import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.UnknownKvStateLocation;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceTypeInfo;
+import org.apache.flink.runtime.util.Hardware;
+import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.Preconditions;
 
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-import akka.dispatch.Recover;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.net.ConnectException;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.CompletableFuture;
 
 /**
- * Client for queryable state.
+ * Client for querying Flink's managed state.
  *
  * <p>You can mark state as queryable via {@link StateDescriptor#setQueryable(String)}.
- * The state instance created from this descriptor will be published for queries
- * when it's created on the TaskManagers and the location will be reported to
- * the JobManager.
+ * The state instance created from this descriptor will be published for queries when it's
+ * created on the Task Managers and the location will be reported to the Job Manager.
  *
- * <p>The client resolves the location of the requested KvState via the
- * JobManager. Resolved locations are cached. When the server address of the
- * requested KvState instance is determined, the client sends out a request to
- * the server.
+ * <p>The client connects to a {@link org.apache.flink.runtime.query.KvStateClientProxy Client Proxy}
+ * running on a given Task Manager. The proxy is the entry point of the client to the Flink cluster.
+ * It forwards the requests of the client to the Job Manager and the required Task Manager, and forwards
+ * the final response back the client.
+ *
+ * <p>The proxy, initially resolves the location of the requested KvState via the JobManager. Resolved
+ * locations are cached. When the server address of the requested KvState instance is determined, the
+ * client sends out a request to the server. The returned final answer is then forwarded to the Client.
  */
+@PublicEvolving
 public class QueryableStateClient {
 
 	private static final Logger LOG = LoggerFactory.getLogger(QueryableStateClient.class);
 
-	/**
-	 * {@link KvStateLocation} lookup to resolve the address of KvState instances.
-	 */
-	private final KvStateLocationLookupService lookupService;
-
-	/**
-	 * Network client for queries against {@link KvStateServer} instances.
-	 */
-	private final KvStateClient kvStateClient;
-
-	/**
-	 * Execution context.
-	 */
-	private final ExecutionContext executionContext;
+	/** The client that forwards the requests to the proxy. */
+	private final Client<KvStateRequest, KvStateResponse> client;
 
-	/**
-	 * Cache for {@link KvStateLocation} instances keyed by job and name.
-	 */
-	private final ConcurrentMap<Tuple2<JobID, String>, Future<KvStateLocation>> lookupCache =
-			new ConcurrentHashMap<>();
-
-	/** This is != null, if we started the actor system. */
-	private final ActorSystem actorSystem;
+	/** The address of the proxy this client is connected to. */
+	private final KvStateServerAddress remoteAddress;
 
+	/** The execution configuration used to instantiate the different (de-)serializers. */
 	private ExecutionConfig executionConfig;
 
 	/**
-	 * Creates a client from the given configuration.
-	 *
-	 * <p>This will create multiple Thread pools: one for the started actor
-	 * system and another for the network client.
-	 *
-	 * @param config Configuration to use.
-	 * @throws Exception Failures are forwarded
+	 * Create the Queryable State Client.
+	 * @param remoteHostname the hostname of the {@link org.apache.flink.runtime.query.KvStateClientProxy proxy}
+	 *                       to connect to.
+	 * @param remotePort the port of the proxy to connect to.
 	 */
-	public QueryableStateClient(Configuration config) throws Exception {
-		this(config, HighAvailabilityServicesUtils.createHighAvailabilityServices(
-				config, Executors.directExecutor(), HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION));
+	public QueryableStateClient(final String remoteHostname, final int remotePort) throws UnknownHostException {
+		this(InetAddress.getByName(Preconditions.checkNotNull(remoteHostname)), remotePort);
 	}
 
 	/**
-	 * Creates a client from the given configuration.
-	 *
-	 * <p>This will create multiple Thread pools: one for the started actor
-	 * system and another for the network client.
-	 *
-	 * @param config Configuration to use.
-	 * @param highAvailabilityServices Service factory for high availability services
-	 * @throws Exception Failures are forwarded
-	 *
-	 * @deprecated This constructor is deprecated and stays only for backwards compatibility. Use the
-	 * {@link #QueryableStateClient(Configuration)} instead.
+	 * Create the Queryable State Client.
+	 * @param remoteAddress the {@link InetAddress address} of the
+	 *                      {@link org.apache.flink.runtime.query.KvStateClientProxy proxy} to connect to.
+	 * @param remotePort the port of the proxy to connect to.
 	 */
-	@Deprecated
-	public QueryableStateClient(
-			Configuration config,
-			HighAvailabilityServices highAvailabilityServices) throws Exception {
-		Preconditions.checkNotNull(config, "Configuration");
-
-		// Create a leader retrieval service
-		LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
-
-		// Get the ask timeout
-		String askTimeoutString = config.getString(AkkaOptions.ASK_TIMEOUT);
-
-		Duration timeout = FiniteDuration.apply(askTimeoutString);
-		if (!timeout.isFinite()) {
-			throw new IllegalConfigurationException(AkkaOptions.ASK_TIMEOUT.key()
-					+ " is not a finite timeout ('" + askTimeoutString + "')");
-		}
-
-		FiniteDuration askTimeout = (FiniteDuration) timeout;
-
-		int lookupRetries = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRIES);
-		int lookupRetryDelayMillis = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRY_DELAY);
-
-		// Retries if no JobManager is around
-		AkkaKvStateLocationLookupService.LookupRetryStrategyFactory retryStrategy =
-				new AkkaKvStateLocationLookupService.FixedDelayLookupRetryStrategyFactory(
-						lookupRetries,
-						FiniteDuration.apply(lookupRetryDelayMillis, "ms"));
+	public QueryableStateClient(final InetAddress remoteAddress, final int remotePort) {
+		Preconditions.checkArgument(remotePort >= 0 && remotePort <= 65536,
+				"Remote Port " + remotePort + " is out of valid port range (0-65536).");
 
-		// Create the actor system
-		@SuppressWarnings("unchecked")
-		Option<Tuple2<String, Object>> remoting = new Some(new Tuple2<>("", 0));
-		this.actorSystem = AkkaUtils.createActorSystem(config, remoting);
+		this.remoteAddress = new KvStateServerAddress(remoteAddress, remotePort);
 
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				actorSystem,
-				askTimeout,
-				retryStrategy);
+		final MessageSerializer<KvStateRequest, KvStateResponse> messageSerializer =
+				new MessageSerializer<>(
+						new KvStateRequest.KvStateRequestDeserializer(),
+						new KvStateResponse.KvStateResponseDeserializer());
 
-		int numEventLoopThreads = config.getInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS);
-
-		if (numEventLoopThreads == 0) {
-			numEventLoopThreads = Runtime.getRuntime().availableProcessors();
-		}
-
-		// Create the network client
-		KvStateClient networkClient = new KvStateClient(
-				numEventLoopThreads,
+		this.client = new Client<>(
+				"Queryable State Client",
+				Hardware.getNumberCPUCores(),
+				messageSerializer,
 				new DisabledKvStateRequestStats());
-
-		this.lookupService = lookupService;
-		this.kvStateClient = networkClient;
-		this.executionContext = actorSystem.dispatcher();
-		this.executionConfig = new ExecutionConfig();
-
-		this.lookupService.start();
-	}
-
-	/** Gets the {@link ExecutionConfig}. */
-	public ExecutionConfig getExecutionConfig() {
-		return executionConfig;
 	}
 
-	/** Sets the {@link ExecutionConfig}. */
-	public void setExecutionConfig(ExecutionConfig config) {
-		this.executionConfig = config;
-	}
-
-	/**
-	 * Creates a client.
-	 *
-	 * @param lookupService    Location lookup service
-	 * @param kvStateClient    Network client for queries
-	 * @param executionContext Execution context for futures
-	 */
-	public QueryableStateClient(
-			KvStateLocationLookupService lookupService,
-			KvStateClient kvStateClient,
-			ExecutionContext executionContext) {
-
-		this.lookupService = Preconditions.checkNotNull(lookupService, "KvStateLocationLookupService");
-		this.kvStateClient = Preconditions.checkNotNull(kvStateClient, "KvStateClient");
-		this.executionContext = Preconditions.checkNotNull(executionContext, "ExecutionContext");
-		this.actorSystem = null;
-
-		this.lookupService.start();
+	/** Shuts down the client. */
+	public void shutdown() {
+		client.shutdown();
 	}
 
 	/**
-	 * Returns the execution context of this client.
-	 *
-	 * @return The execution context used by the client.
+	 * Gets the {@link ExecutionConfig}.
 	 */
-	public ExecutionContext getExecutionContext() {
-		return executionContext;
-	}
-
-	/**
-	 * Shuts down the client and all components.
-	 */
-	public void shutDown() {
-		try {
-			lookupService.shutDown();
-		} catch (Throwable t) {
-			LOG.error("Failed to shut down KvStateLookupService", t);
-		}
-
-		try {
-			kvStateClient.shutDown();
-		} catch (Throwable t) {
-			LOG.error("Failed to shut down KvStateClient", t);
-		}
-
-		if (actorSystem != null) {
-			try {
-				actorSystem.shutdown();
-			} catch (Throwable t) {
-				LOG.error("Failed to shut down ActorSystem", t);
-			}
-		}
+	public ExecutionConfig getExecutionConfig() {
+		return executionConfig;
 	}
 
 	/**
-	 * Returns a future holding the serialized request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
-	 * @param jobId                     JobID of the job the queryable state
-	 *                                  belongs to
-	 * @param queryableStateName        Name under which the state is queryable
-	 * @param keyHashCode               Integer hash code of the key (result of
-	 *                                  a call to {@link Object#hashCode()}
-	 * @param serializedKeyAndNamespace Serialized key and namespace to query
-	 *                                  KvState instance with
-	 * @return Future holding the serialized result
-	 */
-	@SuppressWarnings("unchecked")
-	public Future<byte[]> getKvState(
-			final JobID jobId,
-			final String queryableStateName,
-			final int keyHashCode,
-			final byte[] serializedKeyAndNamespace) {
-
-		return getKvState(jobId, queryableStateName, keyHashCode, serializedKeyAndNamespace, false)
-				.recoverWith(new Recover<Future<byte[]>>() {
-					@Override
-					public Future<byte[]> recover(Throwable failure) throws Throwable {
-						if (failure instanceof UnknownKvStateID ||
-								failure instanceof UnknownKvStateKeyGroupLocation ||
-								failure instanceof UnknownKvStateLocation ||
-								failure instanceof ConnectException) {
-							// These failures are likely to be caused by out-of-sync
-							// KvStateLocation. Therefore we retry this query and
-							// force look up the location.
-							return getKvState(
-									jobId,
-									queryableStateName,
-									keyHashCode,
-									serializedKeyAndNamespace,
-									true);
-						} else {
-							return Futures.failed(failure);
-						}
-					}
-				}, executionContext);
+	 * Replaces the existing {@link ExecutionConfig} (possibly {@code null}), with the provided one.
+	 * @param config The new {@code configuration}.
+	 * @return The old configuration, or {@code null} if none was specified.
+	 * */
+	public ExecutionConfig setExecutionConfig(ExecutionConfig config) {
+		ExecutionConfig prev = executionConfig;
+		this.executionConfig = config;
+		return prev;
 	}
 
 	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
+	 * Returns a future holding the request result.	 *
 	 * @param jobId                     JobID of the job the queryable state belongs to.
 	 * @param queryableStateName        Name under which the state is queryable.
 	 * @param key			            The key we are interested in.
@@ -333,7 +144,7 @@ public class QueryableStateClient {
 	 * @return Future holding the result.
 	 */
 	@PublicEvolving
-	public <K, V> Future<V> getKvState(
+	public <K, V> CompletableFuture<V> getKvState(
 			final JobID jobId,
 			final String queryableStateName,
 			final K key,
@@ -347,16 +158,7 @@ public class QueryableStateClient {
 	}
 
 	/**
-	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
+	 * Returns a future holding the request result.	 *
 	 * @param jobId                     JobID of the job the queryable state belongs to.
 	 * @param queryableStateName        Name under which the state is queryable.
 	 * @param key			            The key we are interested in.
@@ -365,30 +167,19 @@ public class QueryableStateClient {
 	 * @return Future holding the result.
 	 */
 	@PublicEvolving
-	public <K, V> Future<V> getKvState(
+	public <K, V> CompletableFuture<V> getKvState(
 			final JobID jobId,
 			final String queryableStateName,
 			final K key,
 			final TypeInformation<K> keyTypeInfo,
 			final StateDescriptor<?, V> stateDescriptor) {
 
-		Preconditions.checkNotNull(keyTypeInfo);
-
 		return getKvState(jobId, queryableStateName, key, VoidNamespace.INSTANCE,
 				keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor);
 	}
 
 	/**
 	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
 	 * @param jobId                     JobID of the job the queryable state belongs to.
 	 * @param queryableStateName        Name under which the state is queryable.
 	 * @param key			            The key that the state we request is associated with.
@@ -399,7 +190,7 @@ public class QueryableStateClient {
 	 * @return Future holding the result.
 	 */
 	@PublicEvolving
-	public <K, V, N> Future<V> getKvState(
+	public <K, V, N> CompletableFuture<V> getKvState(
 			final JobID jobId,
 			final String queryableStateName,
 			final K key,
@@ -420,15 +211,6 @@ public class QueryableStateClient {
 
 	/**
 	 * Returns a future holding the request result.
-	 *
-	 * <p>If the server does not serve a KvState instance with the given ID,
-	 * the Future will be failed with a {@link UnknownKvStateID}.
-	 *
-	 * <p>If the KvState instance does not hold any data for the given key
-	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
-	 *
-	 * <p>All other failures are forwarded to the Future.
-	 *
 	 * @param jobId                     JobID of the job the queryable state belongs to.
 	 * @param queryableStateName        Name under which the state is queryable.
 	 * @param key			            The key that the state we request is associated with.
@@ -439,7 +221,7 @@ public class QueryableStateClient {
 	 * @return Future holding the result.
 	 */
 	@PublicEvolving
-	public <K, V, N> Future<V> getKvState(
+	public <K, N, V> CompletableFuture<V> getKvState(
 			final JobID jobId,
 			final String queryableStateName,
 			final K key,
@@ -448,8 +230,8 @@ public class QueryableStateClient {
 			final TypeInformation<N> namespaceTypeInfo,
 			final TypeSerializer<V> stateSerializer) {
 
+		Preconditions.checkNotNull(jobId);
 		Preconditions.checkNotNull(queryableStateName);
-
 		Preconditions.checkNotNull(key);
 		Preconditions.checkNotNull(namespace);
 
@@ -457,36 +239,25 @@ public class QueryableStateClient {
 		Preconditions.checkNotNull(namespaceTypeInfo);
 		Preconditions.checkNotNull(stateSerializer);
 
-		if (stateSerializer instanceof ListSerializer) {
-			throw new IllegalArgumentException("ListState is not supported out-of-the-box yet.");
-		}
-
 		TypeSerializer<K> keySerializer = keyTypeInfo.createSerializer(executionConfig);
 		TypeSerializer<N> namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig);
 
 		final byte[] serializedKeyAndNamespace;
 		try {
-			serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
-					key,
-					keySerializer,
-					namespace,
-					namespaceSerializer);
+			serializedKeyAndNamespace = KvStateSerializer
+					.serializeKeyAndNamespace(key, keySerializer, namespace, namespaceSerializer);
 		} catch (IOException e) {
-			return Futures.failed(e);
+			return FutureUtils.getFailedFuture(e);
 		}
 
-		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace)
-				.flatMap(new Mapper<byte[], Future<V>>() {
-					@Override
-					public Future<V> apply(byte[] parameter) {
-						try {
-							return Futures.successful(
-									KvStateSerializer.deserializeValue(parameter, stateSerializer));
-						} catch (IOException e) {
-							return Futures.failed(e);
-						}
+		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace).thenApply(
+				stateResponse -> {
+					try {
+						return KvStateSerializer.deserializeValue(stateResponse.getContent(), stateSerializer);
+					} catch (IOException e) {
+						throw new FlinkRuntimeException(e);
 					}
-				}, executionContext);
+				});
 	}
 
 	/**
@@ -499,92 +270,20 @@ public class QueryableStateClient {
 	 *                                  a call to {@link Object#hashCode()}
 	 * @param serializedKeyAndNamespace Serialized key and namespace to query
 	 *                                  KvState instance with
-	 * @param forceLookup               Flag to force lookup of the {@link KvStateLocation}
 	 * @return Future holding the serialized result
 	 */
-	private Future<byte[]> getKvState(
+	private CompletableFuture<KvStateResponse> getKvState(
 			final JobID jobId,
 			final String queryableStateName,
 			final int keyHashCode,
-			final byte[] serializedKeyAndNamespace,
-			boolean forceLookup) {
-
-		return getKvStateLookupInfo(jobId, queryableStateName, forceLookup)
-				.flatMap(new Mapper<KvStateLocation, Future<byte[]>>() {
-					@Override
-					public Future<byte[]> apply(KvStateLocation lookup) {
-						int keyGroupIndex = KeyGroupRangeAssignment.computeKeyGroupForKeyHash(keyHashCode, lookup.getNumKeyGroups());
-
-						KvStateServerAddress serverAddress = lookup.getKvStateServerAddress(keyGroupIndex);
-						if (serverAddress == null) {
-							return Futures.failed(new UnknownKvStateKeyGroupLocation());
-						} else {
-							// Query server
-							KvStateID kvStateId = lookup.getKvStateID(keyGroupIndex);
-							return kvStateClient.getKvState(serverAddress, kvStateId, serializedKeyAndNamespace);
-						}
-					}
-				}, executionContext);
-	}
-
-	/**
-	 * Lookup the {@link KvStateLocation} for the given job and queryable state
-	 * name.
-	 *
-	 * <p>The job manager will be queried for the location only if forced or no
-	 * cached location can be found. There are no guarantees about
-	 *
-	 * @param jobId              JobID the state instance belongs to.
-	 * @param queryableStateName Name under which the state instance has been published.
-	 * @param forceUpdate        Flag to indicate whether to force a update via the lookup service.
-	 * @return Future holding the KvStateLocation
-	 */
-	private Future<KvStateLocation> getKvStateLookupInfo(
-			JobID jobId,
-			final String queryableStateName,
-			boolean forceUpdate) {
-
-		if (forceUpdate) {
-			Future<KvStateLocation> lookupFuture = lookupService
-					.getKvStateLookupInfo(jobId, queryableStateName);
-			lookupCache.put(new Tuple2<>(jobId, queryableStateName), lookupFuture);
-			return lookupFuture;
-		} else {
-			Tuple2<JobID, String> cacheKey = new Tuple2<>(jobId, queryableStateName);
-			final Future<KvStateLocation> cachedFuture = lookupCache.get(cacheKey);
-
-			if (cachedFuture == null) {
-				Future<KvStateLocation> lookupFuture = lookupService
-						.getKvStateLookupInfo(jobId, queryableStateName);
-
-				Future<KvStateLocation> previous = lookupCache.putIfAbsent(cacheKey, lookupFuture);
-				if (previous == null) {
-					return lookupFuture;
-				} else {
-					return previous;
-				}
-			} else {
-				// do not retain futures which failed as they will remain in
-				// the cache even if the error cause is not present any more
-				// and a new lookup may succeed
-				if (cachedFuture.isCompleted() &&
-						cachedFuture.value().get().isFailure()) {
-					// issue a new lookup
-					Future<KvStateLocation> lookupFuture = lookupService
-							.getKvStateLookupInfo(jobId, queryableStateName);
-
-					// replace the existing one if it has not been replaced yet
-					// otherwise return the one in the cache
-					if (lookupCache.replace(cacheKey, cachedFuture, lookupFuture)) {
-						return lookupFuture;
-					} else {
-						return lookupCache.get(cacheKey);
-					}
-				} else {
-					return cachedFuture;
-				}
-			}
+			final byte[] serializedKeyAndNamespace) {
+		LOG.info("Sending State Request to {}.", remoteAddress);
+		try {
+			KvStateRequest request = new KvStateRequest(jobId, queryableStateName, keyHashCode, serializedKeyAndNamespace);
+			return client.sendRequest(remoteAddress, request);
+		} catch (Exception e) {
+			LOG.error("Unable to send KVStateRequest: ", e);
+			return FutureUtils.getFailedFuture(e);
 		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
new file mode 100644
index 0000000..d7191b6
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyHandler.java
@@ -0,0 +1,225 @@
+/*
+ * 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.queryablestate.client.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.queryablestate.UnknownKvStateIdException;
+import org.apache.flink.queryablestate.UnknownKvStateKeyGroupLocationException;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.AbstractServerHandler;
+import org.apache.flink.queryablestate.network.Client;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.query.KvStateClientProxy;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateMessage;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import scala.concurrent.duration.FiniteDuration;
+import scala.reflect.ClassTag$;
+
+/**
+ * This handler acts as an internal (to the Flink cluster) client that receives
+ * the requests from external clients, executes them by contacting the Job Manager (if necessary) and
+ * the Task Manager holding the requested state, and forwards the answer back to the client.
+ */
+@Internal
+@ChannelHandler.Sharable
+public class KvStateClientProxyHandler extends AbstractServerHandler<KvStateRequest, KvStateResponse> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientProxyHandler.class);
+
+	/** The proxy using this handler. */
+	private final KvStateClientProxy proxy;
+
+	/** A cache to hold the location of different states for which we have already seen requests. */
+	private final ConcurrentMap<Tuple2<JobID, String>, CompletableFuture<KvStateLocation>> lookupCache =
+			new ConcurrentHashMap<>();
+
+	/**
+	 * Network client to forward queries to {@link KvStateServerImpl state server}
+	 * instances inside the cluster.
+	 */
+	private final Client<KvStateInternalRequest, KvStateResponse> kvStateClient;
+
+	/**
+	 * Create the handler used by the {@link KvStateClientProxyImpl}.
+	 *
+	 * @param proxy the {@link KvStateClientProxyImpl proxy} using the handler.
+	 * @param queryExecutorThreads the number of threads used to process incoming requests.
+	 * @param serializer the {@link MessageSerializer} used to (de-) serialize the different messages.
+	 * @param stats server statistics collector.
+	 */
+	public KvStateClientProxyHandler(
+			final KvStateClientProxyImpl proxy,
+			final int queryExecutorThreads,
+			final MessageSerializer<KvStateRequest, KvStateResponse> serializer,
+			final KvStateRequestStats stats) {
+
+		super(proxy, serializer, stats);
+		this.proxy = Preconditions.checkNotNull(proxy);
+		this.kvStateClient = createInternalClient(queryExecutorThreads);
+	}
+
+	private static Client<KvStateInternalRequest, KvStateResponse> createInternalClient(int threads) {
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> messageSerializer =
+				new MessageSerializer<>(
+						new KvStateInternalRequest.KvStateInternalRequestDeserializer(),
+						new KvStateResponse.KvStateResponseDeserializer());
+
+		return new Client<>(
+				"Queryable State Proxy Client",
+				threads,
+				messageSerializer,
+				new DisabledKvStateRequestStats());
+	}
+
+	@Override
+	public CompletableFuture<KvStateResponse> handleRequest(
+			final long requestId,
+			final KvStateRequest request) {
+		CompletableFuture<KvStateResponse> response = new CompletableFuture<>();
+		executeActionAsync(response, request, false);
+		return response;
+	}
+
+	private void executeActionAsync(
+			final CompletableFuture<KvStateResponse> result,
+			final KvStateRequest request,
+			final boolean update) {
+
+		if (!result.isDone()) {
+			final CompletableFuture<KvStateResponse> operationFuture = getState(request, update);
+			operationFuture.whenCompleteAsync(
+					(t, throwable) -> {
+						if (throwable != null) {
+							if (throwable instanceof CancellationException) {
+								result.completeExceptionally(throwable);
+							} else if (throwable.getCause() instanceof UnknownKvStateIdException ||
+									throwable.getCause() instanceof UnknownKvStateKeyGroupLocationException ||
+									throwable.getCause() instanceof UnknownKvStateLocation ||
+									throwable.getCause() instanceof ConnectException) {
+
+								// These failures are likely to be caused by out-of-sync
+								// KvStateLocation. Therefore we retry this query and
+								// force look up the location.
+
+								executeActionAsync(result, request, true);
+							} else {
+								result.completeExceptionally(throwable);
+							}
+						} else {
+							result.complete(t);
+						}
+					}, queryExecutor);
+
+			result.whenComplete(
+					(t, throwable) -> operationFuture.cancel(false));
+		}
+	}
+
+	private CompletableFuture<KvStateResponse> getState(
+			final KvStateRequest request,
+			final boolean forceUpdate) {
+
+		return getKvStateLookupInfo(request.getJobId(), request.getStateName(), forceUpdate)
+				.thenComposeAsync((Function<KvStateLocation, CompletableFuture<KvStateResponse>>) location -> {
+					final int keyGroupIndex = KeyGroupRangeAssignment.computeKeyGroupForKeyHash(
+							request.getKeyHashCode(), location.getNumKeyGroups());
+
+					final KvStateServerAddress serverAddress = location.getKvStateServerAddress(keyGroupIndex);
+					if (serverAddress == null) {
+						return FutureUtils.getFailedFuture(new UnknownKvStateKeyGroupLocationException(getServerName()));
+					} else {
+						// Query server
+						final KvStateID kvStateId = location.getKvStateID(keyGroupIndex);
+						final KvStateInternalRequest internalRequest = new KvStateInternalRequest(
+								kvStateId, request.getSerializedKeyAndNamespace());
+						return kvStateClient.sendRequest(serverAddress, internalRequest);
+					}
+				}, queryExecutor);
+	}
+
+	/**
+	 * Lookup the {@link KvStateLocation} for the given job and queryable state name.
+	 *
+	 * <p>The job manager will be queried for the location only if forced or no
+	 * cached location can be found. There are no guarantees about
+	 *
+	 * @param jobId              JobID the state instance belongs to.
+	 * @param queryableStateName Name under which the state instance has been published.
+	 * @param forceUpdate        Flag to indicate whether to force a update via the lookup service.
+	 * @return Future holding the KvStateLocation
+	 */
+	private CompletableFuture<KvStateLocation> getKvStateLookupInfo(
+			final JobID jobId,
+			final String queryableStateName,
+			final boolean forceUpdate) {
+
+		final Tuple2<JobID, String> cacheKey = new Tuple2<>(jobId, queryableStateName);
+		final CompletableFuture<KvStateLocation> cachedFuture = lookupCache.get(cacheKey);
+
+		if (!forceUpdate && cachedFuture != null && !cachedFuture.isCompletedExceptionally()) {
+			LOG.debug("Retrieving location for state={} of job={} from the cache.", jobId, queryableStateName);
+			return cachedFuture;
+		}
+
+		LOG.debug("Retrieving location for state={} of job={} from the job manager.", jobId, queryableStateName);
+
+		return proxy.getJobManagerFuture().thenComposeAsync(
+				jobManagerGateway -> {
+					final Object msg = new KvStateMessage.LookupKvStateLocation(jobId, queryableStateName);
+					final CompletableFuture<KvStateLocation> locationFuture = FutureUtils.toJava(
+							jobManagerGateway.ask(msg, FiniteDuration.apply(1000L, TimeUnit.MILLISECONDS))
+									.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class)));
+
+					lookupCache.put(cacheKey, locationFuture);
+					return locationFuture;
+				}, queryExecutor);
+	}
+
+	@Override
+	public void shutdown() {
+		kvStateClient.shutdown();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
new file mode 100644
index 0000000..bca80de
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java
@@ -0,0 +1,127 @@
+/*
+ * 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.queryablestate.client.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.UnknownJobManagerException;
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.AbstractServerBase;
+import org.apache.flink.queryablestate.network.AbstractServerHandler;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.query.KvStateClientProxy;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.util.Preconditions;
+
+import java.net.InetAddress;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The default implementation of the {@link KvStateClientProxy}.
+ */
+@Internal
+public class KvStateClientProxyImpl extends AbstractServerBase<KvStateRequest, KvStateResponse> implements KvStateClientProxy {
+
+	private static final CompletableFuture<ActorGateway> UNKNOWN_JOB_MANAGER =
+			FutureUtils.getFailedFuture(new UnknownJobManagerException());
+
+	/** Number of threads used to process incoming requests. */
+	private final int queryExecutorThreads;
+
+	/** Statistics collector. */
+	private final KvStateRequestStats stats;
+
+	private final Object leaderLock = new Object();
+
+	private CompletableFuture<ActorGateway> jobManagerFuture = UNKNOWN_JOB_MANAGER;
+
+	/**
+	 * Creates the Queryable State Client Proxy.
+	 *
+	 * <p>The server is instantiated using reflection by the
+	 * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateClientProxy(InetAddress, int, int, int, KvStateRequestStats)
+	 * QueryableStateUtils.startKvStateClientProxy(InetAddress, int, int, int, KvStateRequestStats)}.
+	 *
+	 * <p>The server needs to be started via {@link #start()} in order to bind
+	 * to the configured bind address.
+	 *
+	 * @param bindAddress the address to listen to.
+	 * @param bindPort the port to listen to.
+	 * @param numEventLoopThreads number of event loop threads.
+	 * @param numQueryThreads number of query threads.
+	 * @param stats the statistics collector.
+	 */
+	public KvStateClientProxyImpl(
+			final InetAddress bindAddress,
+			final Integer bindPort,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads,
+			final KvStateRequestStats stats) {
+
+		super("Queryable State Proxy Server", bindAddress, bindPort, numEventLoopThreads, numQueryThreads);
+		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
+		this.queryExecutorThreads = numQueryThreads;
+		this.stats = Preconditions.checkNotNull(stats);
+	}
+
+	@Override
+	public KvStateServerAddress getServerAddress() {
+		return super.getServerAddress();
+	}
+
+	@Override
+	public void start() throws InterruptedException {
+		super.start();
+	}
+
+	@Override
+	public void shutdown() {
+		super.shutdown();
+	}
+
+	@Override
+	public void updateJobManager(CompletableFuture<ActorGateway> leadingJobManager) throws Exception {
+		synchronized (leaderLock) {
+			if (leadingJobManager == null) {
+				jobManagerFuture = UNKNOWN_JOB_MANAGER;
+			} else {
+				jobManagerFuture = leadingJobManager;
+			}
+		}
+	}
+
+	@Override
+	public CompletableFuture<ActorGateway> getJobManagerFuture() {
+		synchronized (leaderLock) {
+			return jobManagerFuture;
+		}
+	}
+
+	@Override
+	public AbstractServerHandler<KvStateRequest, KvStateResponse> initializeHandler() {
+		MessageSerializer<KvStateRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(
+						new KvStateRequest.KvStateRequestDeserializer(),
+						new KvStateResponse.KvStateResponseDeserializer());
+		return new KvStateClientProxyHandler(this, queryExecutorThreads, serializer, stats);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
new file mode 100644
index 0000000..eedc2a1
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateInternalRequest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.queryablestate.messages;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageDeserializer;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import java.nio.ByteBuffer;
+
+/**
+ * The request to be forwarded by the {@link org.apache.flink.runtime.query.KvStateClientProxy
+ * Queryable State Client Proxy} to the {@link org.apache.flink.runtime.query.KvStateServer State Server}
+ * of the Task Manager responsible for the requested state.
+ */
+@Internal
+public class KvStateInternalRequest extends MessageBody {
+
+	private final KvStateID kvStateId;
+	private final byte[] serializedKeyAndNamespace;
+
+	public KvStateInternalRequest(
+			final KvStateID stateId,
+			final byte[] serializedKeyAndNamespace) {
+
+		this.kvStateId = Preconditions.checkNotNull(stateId);
+		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace);
+	}
+
+	public KvStateID getKvStateId() {
+		return kvStateId;
+	}
+
+	public byte[] getSerializedKeyAndNamespace() {
+		return serializedKeyAndNamespace;
+	}
+
+	@Override
+	public byte[] serialize() {
+
+		// KvStateId + sizeOf(serializedKeyAndNamespace) + serializedKeyAndNamespace
+		final int size = KvStateID.SIZE + Integer.BYTES + serializedKeyAndNamespace.length;
+
+		return ByteBuffer.allocate(size)
+				.putLong(kvStateId.getLowerPart())
+				.putLong(kvStateId.getUpperPart())
+				.putInt(serializedKeyAndNamespace.length)
+				.put(serializedKeyAndNamespace)
+				.array();
+	}
+
+	/**
+	 * A {@link MessageDeserializer deserializer} for {@link KvStateInternalRequest}.
+	 */
+	public static class KvStateInternalRequestDeserializer implements MessageDeserializer<KvStateInternalRequest> {
+
+		@Override
+		public KvStateInternalRequest deserializeMessage(ByteBuf buf) {
+			KvStateID kvStateId = new KvStateID(buf.readLong(), buf.readLong());
+
+			int length = buf.readInt();
+			Preconditions.checkArgument(length >= 0,
+					"Negative length for key and namespace. " +
+							"This indicates a serialization error.");
+
+			byte[] serializedKeyAndNamespace = new byte[length];
+			if (length > 0) {
+				buf.readBytes(serializedKeyAndNamespace);
+			}
+			return new KvStateInternalRequest(kvStateId, serializedKeyAndNamespace);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
index eb33bce..7eb39c7 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
@@ -18,72 +18,124 @@
 
 package org.apache.flink.queryablestate.messages;
 
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageDeserializer;
 import org.apache.flink.util.Preconditions;
 
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
 /**
- * A {@link InternalKvState} instance request for a specific key and namespace.
+ * The request to be sent by the {@link org.apache.flink.queryablestate.client.QueryableStateClient
+ * Queryable State Client} to the {@link org.apache.flink.runtime.query.KvStateClientProxy Client Proxy}
+ * requesting a given state.
  */
-public final class KvStateRequest {
+@Internal
+public class KvStateRequest extends MessageBody {
 
-	/** ID for this request. */
-	private final long requestId;
+	private final JobID jobId;
+	private final String stateName;
+	private final int keyHashCode;
+	private final byte[] serializedKeyAndNamespace;
 
-	/** ID of the requested KvState instance. */
-	private final KvStateID kvStateId;
+	public KvStateRequest(
+			final JobID jobId,
+			final String stateName,
+			final int keyHashCode,
+			final byte[] serializedKeyAndNamespace) {
 
-	/** Serialized key and namespace to request from the KvState instance. */
-	private final byte[] serializedKeyAndNamespace;
+		this.jobId = Preconditions.checkNotNull(jobId);
+		this.stateName = Preconditions.checkNotNull(stateName);
+		this.keyHashCode = keyHashCode;
+		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace);
+	}
 
-	/**
-	 * Creates a KvState instance request.
-	 *
-	 * @param requestId                 ID for this request
-	 * @param kvStateId                 ID of the requested KvState instance
-	 * @param serializedKeyAndNamespace Serialized key and namespace to request from the KvState
-	 *                                  instance
-	 */
-	public KvStateRequest(long requestId, KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-		this.requestId = requestId;
-		this.kvStateId = Preconditions.checkNotNull(kvStateId, "KvStateID");
-		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
+	public JobID getJobId() {
+		return jobId;
 	}
 
-	/**
-	 * Returns the request ID.
-	 *
-	 * @return Request ID
-	 */
-	public long getRequestId() {
-		return requestId;
+	public String getStateName() {
+		return stateName;
 	}
 
-	/**
-	 * Returns the ID of the requested KvState instance.
-	 *
-	 * @return ID of the requested KvState instance
-	 */
-	public KvStateID getKvStateId() {
-		return kvStateId;
+	public int getKeyHashCode() {
+		return keyHashCode;
 	}
 
-	/**
-	 * Returns the serialized key and namespace to request from the KvState
-	 * instance.
-	 *
-	 * @return Serialized key and namespace to request from the KvState instance
-	 */
 	public byte[] getSerializedKeyAndNamespace() {
 		return serializedKeyAndNamespace;
 	}
 
 	@Override
+	public byte[] serialize() {
+
+		byte[] serializedStateName = stateName.getBytes();
+
+		// JobID + stateName + sizeOf(stateName) + hashCode + keyAndNamespace + sizeOf(keyAndNamespace)
+		final int size =
+				JobID.SIZE +
+				serializedStateName.length + Integer.BYTES +
+				Integer.BYTES +
+				serializedKeyAndNamespace.length + Integer.BYTES;
+
+		return ByteBuffer.allocate(size)
+				.putLong(jobId.getLowerPart())
+				.putLong(jobId.getUpperPart())
+				.putInt(serializedStateName.length)
+				.put(serializedStateName)
+				.putInt(keyHashCode)
+				.putInt(serializedKeyAndNamespace.length)
+				.put(serializedKeyAndNamespace)
+				.array();
+	}
+
+	@Override
 	public String toString() {
 		return "KvStateRequest{" +
-				"requestId=" + requestId +
-				", kvStateId=" + kvStateId +
-				", serializedKeyAndNamespace.length=" + serializedKeyAndNamespace.length +
+				"jobId=" + jobId +
+				", stateName='" + stateName + '\'' +
+				", keyHashCode=" + keyHashCode +
+				", serializedKeyAndNamespace=" + Arrays.toString(serializedKeyAndNamespace) +
 				'}';
 	}
+
+	/**
+	 * A {@link MessageDeserializer deserializer} for {@link KvStateRequest}.
+	 */
+	public static class KvStateRequestDeserializer implements MessageDeserializer<KvStateRequest> {
+
+		@Override
+		public KvStateRequest deserializeMessage(ByteBuf buf) {
+			JobID jobId = new JobID(buf.readLong(), buf.readLong());
+
+			int statenameLength = buf.readInt();
+			Preconditions.checkArgument(statenameLength >= 0,
+					"Negative length for state name. " +
+							"This indicates a serialization error.");
+
+			String stateName = "";
+			if (statenameLength > 0) {
+				byte[] name = new byte[statenameLength];
+				buf.readBytes(name);
+				stateName = new String(name);
+			}
+
+			int keyHashCode = buf.readInt();
+
+			int knamespaceLength = buf.readInt();
+			Preconditions.checkArgument(knamespaceLength >= 0,
+					"Negative length for key and namespace. " +
+							"This indicates a serialization error.");
+
+			byte[] serializedKeyAndNamespace = new byte[knamespaceLength];
+			if (knamespaceLength > 0) {
+				buf.readBytes(serializedKeyAndNamespace);
+			}
+			return new KvStateRequest(jobId, stateName, keyHashCode, serializedKeyAndNamespace);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
deleted file mode 100644
index 4015d79..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
+++ /dev/null
@@ -1,68 +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.queryablestate.messages;
-
-/**
- * A failure response to a {@link KvStateRequest}.
- */
-public final class KvStateRequestFailure {
-
-	/** ID of the request responding to. */
-	private final long requestId;
-
-	/** Failure cause. Not allowed to be a user type. */
-	private final Throwable cause;
-
-	/**
-	 * Creates a failure response to a {@link KvStateRequest}.
-	 *
-	 * @param requestId ID for the request responding to
-	 * @param cause     Failure cause (not allowed to be a user type)
-	 */
-	public KvStateRequestFailure(long requestId, Throwable cause) {
-		this.requestId = requestId;
-		this.cause = cause;
-	}
-
-	/**
-	 * Returns the request ID responding to.
-	 *
-	 * @return Request ID responding to
-	 */
-	public long getRequestId() {
-		return requestId;
-	}
-
-	/**
-	 * Returns the failure cause.
-	 *
-	 * @return Failure cause
-	 */
-	public Throwable getCause() {
-		return cause;
-	}
-
-	@Override
-	public String toString() {
-		return "KvStateRequestFailure{" +
-				"requestId=" + requestId +
-				", cause=" + cause +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
deleted file mode 100644
index 6bf2397..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
+++ /dev/null
@@ -1,74 +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.queryablestate.messages;
-
-import org.apache.flink.util.Preconditions;
-
-/**
- * A successful response to a {@link KvStateRequest} containing the serialized
- * result for the requested key and namespace.
- */
-public final class KvStateRequestResult {
-
-	/** ID of the request responding to. */
-	private final long requestId;
-
-	/**
-	 * Serialized result for the requested key and namespace. If no result was
-	 * available for the specified key and namespace, this is <code>null</code>.
-	 */
-	private final byte[] serializedResult;
-
-	/**
-	 * Creates a successful {@link KvStateRequestResult} response.
-	 *
-	 * @param requestId        ID of the request responding to
-	 * @param serializedResult Serialized result or <code>null</code> if none
-	 */
-	public KvStateRequestResult(long requestId, byte[] serializedResult) {
-		this.requestId = requestId;
-		this.serializedResult = Preconditions.checkNotNull(serializedResult, "Serialization result");
-	}
-
-	/**
-	 * Returns the request ID responding to.
-	 *
-	 * @return Request ID responding to
-	 */
-	public long getRequestId() {
-		return requestId;
-	}
-
-	/**
-	 * Returns the serialized result or <code>null</code> if none available.
-	 *
-	 * @return Serialized result or <code>null</code> if none available.
-	 */
-	public byte[] getSerializedResult() {
-		return serializedResult;
-	}
-
-	@Override
-	public String toString() {
-		return "KvStateRequestResult{" +
-				"requestId=" + requestId +
-				", serializedResult.length=" + serializedResult.length +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
new file mode 100644
index 0000000..462135f
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateResponse.java
@@ -0,0 +1,75 @@
+/*
+ * 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.queryablestate.messages;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageDeserializer;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import java.nio.ByteBuffer;
+
+/**
+ * The response containing the (serialized) state sent by the {@link org.apache.flink.runtime.query.KvStateServer
+ * State Server} to the {@link org.apache.flink.runtime.query.KvStateClientProxy Client Proxy}, and then forwarded
+ * by the proxy to the original {@link org.apache.flink.queryablestate.client.QueryableStateClient Queryable State
+ * Client}.
+ */
+@Internal
+public class KvStateResponse extends MessageBody {
+
+	private final byte[] content;
+
+	public KvStateResponse(final byte[] content) {
+		this.content = Preconditions.checkNotNull(content);
+	}
+
+	public byte[] getContent() {
+		return content;
+	}
+
+	@Override
+	public byte[] serialize() {
+		final int size = Integer.BYTES + content.length;
+		return ByteBuffer.allocate(size)
+				.putInt(content.length)
+				.put(content)
+				.array();
+	}
+
+	/**
+	 * A {@link MessageDeserializer deserializer} for {@link KvStateResponseDeserializer}.
+	 */
+	public static class KvStateResponseDeserializer implements MessageDeserializer<KvStateResponse> {
+
+		@Override
+		public KvStateResponse deserializeMessage(ByteBuf buf) {
+			int length = buf.readInt();
+			Preconditions.checkArgument(length >= 0,
+					"Negative length for state content. " +
+							"This indicates a serialization error.");
+			byte[] content = new byte[length];
+			buf.readBytes(content);
+
+			return new KvStateResponse(content);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
new file mode 100644
index 0000000..4bf8e98
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java
@@ -0,0 +1,241 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The base class for every server in the queryable state module.
+ * It is using pure netty to send and receive messages of type {@link MessageBody}.
+ *
+ * @param <REQ> the type of request the server expects to receive.
+ * @param <RESP> the type of response the server will send.
+ */
+@Internal
+public abstract class AbstractServerBase<REQ extends MessageBody, RESP extends MessageBody> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(AbstractServerBase.class);
+
+	/** AbstractServerBase config: low water mark. */
+	private static final int LOW_WATER_MARK = 8 * 1024;
+
+	/** AbstractServerBase config: high water mark. */
+	private static final int HIGH_WATER_MARK = 32 * 1024;
+
+	private final String serverName;
+
+	/** Netty's ServerBootstrap. */
+	private final ServerBootstrap bootstrap;
+
+	/** Query executor thread pool. */
+	private final ExecutorService queryExecutor;
+
+	/** Address of this server. */
+	private KvStateServerAddress serverAddress;
+
+	/** The handler used for the incoming messages. */
+	private AbstractServerHandler<REQ, RESP> handler;
+
+	/**
+	 * Creates the {@link AbstractServerBase}.
+	 *
+	 * <p>The server needs to be started via {@link #start()} in order to bind
+	 * to the configured bind address.
+	 *
+	 * @param serverName the name of the server
+	 * @param bindAddress address to bind to
+	 * @param bindPort port to bind to (random port if 0)
+	 * @param numEventLoopThreads number of event loop threads
+	 */
+	protected AbstractServerBase(
+			final String serverName,
+			final InetAddress bindAddress,
+			final Integer bindPort,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads) {
+
+		Preconditions.checkNotNull(bindAddress);
+		Preconditions.checkArgument(bindPort >= 0 && bindPort <= 65536, "Port " + bindPort + " out of valid range (0-65536).");
+		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
+		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
+
+		this.serverName = Preconditions.checkNotNull(serverName);
+		this.queryExecutor = createQueryExecutor(numQueryThreads);
+
+		final NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
+
+		final ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink " + serverName + " EventLoop Thread %d")
+				.build();
+
+		final NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
+
+		bootstrap = new ServerBootstrap()
+				// Bind address and port
+				.localAddress(bindAddress, bindPort)
+				// NIO server channels
+				.group(nioGroup)
+				.channel(NioServerSocketChannel.class)
+				// AbstractServerBase channel Options
+				.option(ChannelOption.ALLOCATOR, bufferPool)
+				// Child channel options
+				.childOption(ChannelOption.ALLOCATOR, bufferPool)
+				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
+				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK);
+	}
+
+	/**
+	 * Creates a thread pool for the query execution.
+	 *
+	 * @param numQueryThreads Number of query threads.
+	 * @return Thread pool for query execution
+	 */
+	private ExecutorService createQueryExecutor(int numQueryThreads) {
+		ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink " + getServerName() + " Thread %d")
+				.build();
+
+		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
+	}
+
+	protected ExecutorService getQueryExecutor() {
+		return queryExecutor;
+	}
+
+	public String getServerName() {
+		return serverName;
+	}
+
+	public abstract AbstractServerHandler<REQ, RESP> initializeHandler();
+
+	/**
+	 * Starts the server by binding to the configured bind address (blocking).
+	 * @throws InterruptedException If interrupted during the bind operation
+	 */
+	public void start() throws InterruptedException {
+		Preconditions.checkState(serverAddress == null,
+				"Server " + serverName + " has already been started @ " + serverAddress + '.');
+
+		this.handler = initializeHandler();
+		bootstrap.childHandler(new ServerChannelInitializer<>(handler));
+
+		Channel channel = bootstrap.bind().sync().channel();
+		InetSocketAddress localAddress = (InetSocketAddress) channel.localAddress();
+		serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
+
+		LOG.info("Started server {} @ {}", serverName, serverAddress);
+	}
+
+	/**
+	 * Returns the address of this server.
+	 *
+	 * @return AbstractServerBase address
+	 * @throws IllegalStateException If server has not been started yet
+	 */
+	public KvStateServerAddress getServerAddress() {
+		Preconditions.checkState(serverAddress != null, "Server " + serverName + " has not been started.");
+		return serverAddress;
+	}
+
+	/**
+	 * Shuts down the server and all related thread pools.
+	 */
+	public void shutdown() {
+		LOG.info("Shutting down server {} @ {}", serverName, serverAddress);
+
+		if (handler != null) {
+			handler.shutdown();
+		}
+
+		if (queryExecutor != null) {
+			queryExecutor.shutdown();
+		}
+
+		if (bootstrap != null) {
+			EventLoopGroup group = bootstrap.group();
+			if (group != null) {
+				group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS);
+			}
+		}
+		serverAddress = null;
+	}
+
+	/**
+	 * Channel pipeline initializer.
+	 *
+	 * <p>The request handler is shared, whereas the other handlers are created
+	 * per channel.
+	 */
+	private static final class ServerChannelInitializer<REQ extends MessageBody, RESP extends MessageBody> extends ChannelInitializer<SocketChannel> {
+
+		/** The shared request handler. */
+		private final AbstractServerHandler<REQ, RESP> sharedRequestHandler;
+
+		/**
+		 * Creates the channel pipeline initializer with the shared request handler.
+		 *
+		 * @param sharedRequestHandler Shared request handler.
+		 */
+		ServerChannelInitializer(AbstractServerHandler<REQ, RESP> sharedRequestHandler) {
+			this.sharedRequestHandler = Preconditions.checkNotNull(sharedRequestHandler, "MessageBody handler");
+		}
+
+		@Override
+		protected void initChannel(SocketChannel channel) throws Exception {
+			channel.pipeline()
+					.addLast(new ChunkedWriteHandler())
+					.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+					.addLast(sharedRequestHandler);
+		}
+	}
+
+	@VisibleForTesting
+	public boolean isExecutorShutdown() {
+		return queryExecutor.isShutdown();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
new file mode 100644
index 0000000..b9bf671
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerHandler.java
@@ -0,0 +1,306 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The base class of every handler used by an {@link AbstractServerBase}.
+ *
+ * @param <REQ> the type of request the server expects to receive.
+ * @param <RESP> the type of response the server will send.
+ */
+@Internal
+@ChannelHandler.Sharable
+public abstract class AbstractServerHandler<REQ extends MessageBody, RESP extends MessageBody> extends ChannelInboundHandlerAdapter {
+
+	private static final Logger LOG = LoggerFactory.getLogger(AbstractServerHandler.class);
+
+	/** The owning server of this handler. */
+	private final AbstractServerBase<REQ, RESP> server;
+
+	/** The serializer used to (de-)serialize messages. */
+	private final MessageSerializer<REQ, RESP> serializer;
+
+	/** Thread pool for query execution. */
+	protected final ExecutorService queryExecutor;
+
+	/** Exposed server statistics. */
+	private final KvStateRequestStats stats;
+
+	/**
+	 * Create the handler.
+	 *
+	 * @param serializer the serializer used to (de-)serialize messages
+	 * @param stats statistics collector
+	 */
+	public AbstractServerHandler(
+			final AbstractServerBase<REQ, RESP> server,
+			final MessageSerializer<REQ, RESP> serializer,
+			final KvStateRequestStats stats) {
+
+		this.server = Preconditions.checkNotNull(server);
+		this.serializer = Preconditions.checkNotNull(serializer);
+		this.queryExecutor = server.getQueryExecutor();
+		this.stats = Preconditions.checkNotNull(stats);
+	}
+
+	protected String getServerName() {
+		return server.getServerName();
+	}
+
+	@Override
+	public void channelActive(ChannelHandlerContext ctx) throws Exception {
+		stats.reportActiveConnection();
+	}
+
+	@Override
+	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+		stats.reportInactiveConnection();
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		REQ request = null;
+		long requestId = -1L;
+
+		try {
+			final ByteBuf buf = (ByteBuf) msg;
+			final MessageType msgType = MessageSerializer.deserializeHeader(buf);
+
+			requestId = MessageSerializer.getRequestId(buf);
+
+			if (msgType == MessageType.REQUEST) {
+
+				// ------------------------------------------------------------
+				// MessageBody
+				// ------------------------------------------------------------
+				request = serializer.deserializeRequest(buf);
+				stats.reportRequest();
+
+				// Execute actual query async, because it is possibly
+				// blocking (e.g. file I/O).
+				//
+				// A submission failure is not treated as fatal. todo here if there is a shared resource e.g. registry, then I will have to sync on that.
+				queryExecutor.submit(new AsyncRequestTask<>(this, ctx, requestId, request, stats));
+
+			} else {
+				// ------------------------------------------------------------
+				// Unexpected
+				// ------------------------------------------------------------
+
+				final String errMsg = "Unexpected message type " + msgType + ". Expected " + MessageType.REQUEST + ".";
+				final ByteBuf failure = MessageSerializer.serializeServerFailure(ctx.alloc(), new IllegalArgumentException(errMsg));
+
+				LOG.debug(errMsg);
+				ctx.writeAndFlush(failure);
+			}
+		} catch (Throwable t) {
+			final String stringifiedCause = ExceptionUtils.stringifyException(t);
+
+			String errMsg;
+			ByteBuf err;
+			if (request != null) {
+				errMsg = "Failed request with ID " + requestId + ". Caused by: " + stringifiedCause;
+				err = MessageSerializer.serializeRequestFailure(ctx.alloc(), requestId, new RuntimeException(errMsg));
+				stats.reportFailedRequest();
+			} else {
+				errMsg = "Failed incoming message. Caused by: " + stringifiedCause;
+				err = MessageSerializer.serializeServerFailure(ctx.alloc(), new RuntimeException(errMsg));
+			}
+
+			LOG.debug(errMsg);
+			ctx.writeAndFlush(err);
+
+		} finally {
+			// IMPORTANT: We have to always recycle the incoming buffer.
+			// Otherwise we will leak memory out of Netty's buffer pool.
+			//
+			// If any operation ever holds on to the buffer, it is the
+			// responsibility of that operation to retain the buffer and
+			// release it later.
+			ReferenceCountUtil.release(msg);
+		}
+	}
+
+	@Override
+	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+		final String msg = "Exception in server pipeline. Caused by: " + ExceptionUtils.stringifyException(cause);
+		final ByteBuf err = serializer.serializeServerFailure(ctx.alloc(), new RuntimeException(msg));
+
+		LOG.debug(msg);
+		ctx.writeAndFlush(err).addListener(ChannelFutureListener.CLOSE);
+	}
+
+	/**
+	 * Handles an incoming request and returns a {@link CompletableFuture} containing the corresponding response.
+	 *
+	 * <p><b>NOTE:</b> This method is called by multiple threads.
+	 *
+	 * @param requestId the id of the received request to be handled.
+	 * @param request the request to be handled.
+	 * @return A future with the response to be forwarded to the client.
+	 */
+	public abstract CompletableFuture<RESP> handleRequest(final long requestId, final REQ request);
+
+	/**
+	 * Shuts down any handler specific resources, e.g. thread pools etc.
+	 */
+	public abstract void shutdown();
+
+	/**
+	 * Task to execute the actual query against the {@link InternalKvState} instance.
+	 */
+	private static class AsyncRequestTask<REQ extends MessageBody, RESP extends MessageBody> implements Runnable {
+
+		private final AbstractServerHandler<REQ, RESP> handler;
+
+		private final ChannelHandlerContext ctx;
+
+		private final long requestId;
+
+		private final REQ request;
+
+		private final KvStateRequestStats stats;
+
+		private final long creationNanos;
+
+		AsyncRequestTask(
+				final AbstractServerHandler<REQ, RESP> handler,
+				final ChannelHandlerContext ctx,
+				final long requestId,
+				final REQ request,
+				final KvStateRequestStats stats) {
+
+			this.handler = Preconditions.checkNotNull(handler);
+			this.ctx = Preconditions.checkNotNull(ctx);
+			this.requestId = requestId;
+			this.request = Preconditions.checkNotNull(request);
+			this.stats = Preconditions.checkNotNull(stats);
+			this.creationNanos = System.nanoTime();
+		}
+
+		@Override
+		public void run() {
+
+			if (!ctx.channel().isActive()) {
+				return;
+			}
+
+			handler.handleRequest(requestId, request).whenComplete((resp, throwable) -> {
+				try {
+					if (throwable != null) {
+						throw throwable instanceof CompletionException
+								? throwable.getCause()
+								: throwable;
+					}
+
+					if (resp == null) {
+						throw new BadRequestException(handler.getServerName(), "NULL returned for request with ID " + requestId + ".");
+					}
+
+					final ByteBuf serialResp = MessageSerializer.serializeResponse(ctx.alloc(), requestId, resp);
+
+					int highWatermark = ctx.channel().config().getWriteBufferHighWaterMark();
+
+					ChannelFuture write;
+					if (serialResp.readableBytes() <= highWatermark) {
+						write = ctx.writeAndFlush(serialResp);
+					} else {
+						write = ctx.writeAndFlush(new ChunkedByteBuf(serialResp, highWatermark));
+					}
+					write.addListener(new RequestWriteListener());
+
+				} catch (BadRequestException e) {
+					try {
+						stats.reportFailedRequest();
+						final ByteBuf err = MessageSerializer.serializeRequestFailure(ctx.alloc(), requestId, e);
+						ctx.writeAndFlush(err);
+					} catch (IOException io) {
+						LOG.error("Failed to respond with the error after failed request", io);
+					}
+				} catch (Throwable t) {
+					try {
+						stats.reportFailedRequest();
+
+						final String errMsg = "Failed request " + requestId + ". Caused by: " + ExceptionUtils.stringifyException(t);
+						final ByteBuf err = MessageSerializer.serializeRequestFailure(ctx.alloc(), requestId, new RuntimeException(errMsg));
+						ctx.writeAndFlush(err);
+					} catch (IOException io) {
+						LOG.error("Failed to respond with the error after failed request", io);
+					}
+				}
+			});
+		}
+
+		@Override
+		public String toString() {
+			return "AsyncRequestTask{" +
+					"requestId=" + requestId +
+					", request=" + request +
+					'}';
+		}
+
+		/**
+		 * Callback after query result has been written.
+		 *
+		 * <p>Gathers stats and logs errors.
+		 */
+		private class RequestWriteListener implements ChannelFutureListener {
+
+			@Override
+			public void operationComplete(ChannelFuture future) throws Exception {
+				long durationNanos = System.nanoTime() - creationNanos;
+				long durationMillis = TimeUnit.MILLISECONDS.convert(durationNanos, TimeUnit.NANOSECONDS);
+
+				if (future.isSuccess()) {
+					LOG.debug("Request {} was successfully answered after {} ms.", request, durationMillis);
+					stats.reportSuccessfulRequest(durationMillis);
+				} else {
+					LOG.debug("Request {} failed after {} ms : ", request, durationMillis, future.cause());
+					stats.reportFailedRequest();
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java
new file mode 100644
index 0000000..3c0c484
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/BadRequestException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Base class for exceptions thrown during querying Flink's managed state.
+ */
+@Internal
+public class BadRequestException extends Exception {
+
+	private static final long serialVersionUID = 3458743952407632903L;
+
+	public BadRequestException(String serverName, String message) {
+		super(Preconditions.checkNotNull(serverName) + " : " + message);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java
new file mode 100644
index 0000000..9c56025
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ChunkedByteBuf.java
@@ -0,0 +1,100 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+/**
+ * A {@link ByteBuf} instance to be consumed in chunks by {@link ChunkedWriteHandler},
+ * respecting the high and low watermarks.
+ *
+ * @see <a href="http://normanmaurer.me/presentations/2014-facebook-eng-netty/slides.html#10.0">Low/High Watermarks</a>
+ */
+@Internal
+public class ChunkedByteBuf implements ChunkedInput<ByteBuf> {
+
+	/** The buffer to chunk. */
+	private final ByteBuf buf;
+
+	/** Size of chunks. */
+	private final int chunkSize;
+
+	/** Closed flag. */
+	private boolean isClosed;
+
+	/** End of input flag. */
+	private boolean isEndOfInput;
+
+	public ChunkedByteBuf(ByteBuf buf, int chunkSize) {
+		this.buf = Preconditions.checkNotNull(buf, "Buffer");
+		Preconditions.checkArgument(chunkSize > 0, "Non-positive chunk size");
+		this.chunkSize = chunkSize;
+	}
+
+	@Override
+	public boolean isEndOfInput() throws Exception {
+		return isClosed || isEndOfInput;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!isClosed) {
+			// If we did not consume the whole buffer yet, we have to release
+			// it here. Otherwise, it's the responsibility of the consumer.
+			if (!isEndOfInput) {
+				buf.release();
+			}
+
+			isClosed = true;
+		}
+	}
+
+	@Override
+	public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception {
+		if (isClosed) {
+			return null;
+		} else if (buf.readableBytes() <= chunkSize) {
+			isEndOfInput = true;
+
+			// Don't retain as the consumer is responsible to release it
+			return buf.slice();
+		} else {
+			// Return a chunk sized slice of the buffer. The ref count is
+			// shared with the original buffer. That's why we need to retain
+			// a reference here.
+			return buf.readSlice(chunkSize).retain();
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "ChunkedByteBuf{" +
+				"buf=" + buf +
+				", chunkSize=" + chunkSize +
+				", isClosed=" + isClosed +
+				", isEndOfInput=" + isEndOfInput +
+				'}';
+	}
+}


[09/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
deleted file mode 100644
index 5d4a861..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
+++ /dev/null
@@ -1,458 +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.queryablestate.network;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.queryablestate.UnknownKvStateID;
-import org.apache.flink.queryablestate.UnknownKvStateKeyGroupLocation;
-import org.apache.flink.queryablestate.client.KvStateClient;
-import org.apache.flink.queryablestate.client.KvStateLocationLookupService;
-import org.apache.flink.queryablestate.client.QueryableStateClient;
-import org.apache.flink.queryablestate.server.KvStateServerImpl;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateLocation;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServer;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.UnknownKvStateLocation;
-import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
-import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.heap.HeapValueState;
-import org.apache.flink.runtime.state.heap.NestedMapsStateTable;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.util.MathUtils;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link QueryableStateClient}.
- */
-public class QueryableStateClientTest {
-
-	private static final ActorSystem testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-
-	private static final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS);
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
-	}
-
-	/**
-	 * All failures should lead to a retry with a forced location lookup.
-	 *
-	 * <p>UnknownKvStateID, UnknownKvStateKeyGroupLocation, UnknownKvStateLocation,
-	 * ConnectException are checked explicitly as these indicate out-of-sync
-	 * KvStateLocation.
-	 */
-	@Test
-	public void testForceLookupOnOutdatedLocation() throws Exception {
-		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-		KvStateClient networkClient = mock(KvStateClient.class);
-
-		QueryableStateClient client = new QueryableStateClient(
-				lookupService,
-				networkClient,
-				testActorSystem.dispatcher());
-
-		try {
-			JobID jobId = new JobID();
-			int numKeyGroups = 4;
-
-			//
-			// UnknownKvStateLocation
-			//
-			String query1 = "lucky";
-
-			Future<KvStateLocation> unknownKvStateLocation = Futures.failed(
-					new UnknownKvStateLocation(query1));
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query1)))
-					.thenReturn(unknownKvStateLocation);
-
-			Future<Integer> result = client.getKvState(
-					jobId,
-					query1,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateLocation exception");
-			} catch (UnknownKvStateLocation ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query1));
-
-			//
-			// UnknownKvStateKeyGroupLocation
-			//
-			String query2 = "unlucky";
-
-			Future<KvStateLocation> unknownKeyGroupLocation = Futures.successful(
-					new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query2));
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query2)))
-					.thenReturn(unknownKeyGroupLocation);
-
-			result = client.getKvState(
-					jobId,
-					query2,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateKeyGroupLocation exception");
-			} catch (UnknownKvStateKeyGroupLocation ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query2));
-
-			//
-			// UnknownKvStateID
-			//
-			String query3 = "water";
-			KvStateID kvStateId = new KvStateID();
-			Future<byte[]> unknownKvStateId = Futures.failed(new UnknownKvStateID(kvStateId));
-
-			KvStateServerAddress serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 12323);
-			KvStateLocation location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query3);
-			for (int i = 0; i < numKeyGroups; i++) {
-				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
-			}
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query3)))
-					.thenReturn(Futures.successful(location));
-
-			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
-					.thenReturn(unknownKvStateId);
-
-			result = client.getKvState(
-					jobId,
-					query3,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateID exception");
-			} catch (UnknownKvStateID ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query3));
-
-			//
-			// ConnectException
-			//
-			String query4 = "space";
-			Future<byte[]> connectException = Futures.failed(new ConnectException());
-			kvStateId = new KvStateID();
-
-			serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 11123);
-			location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query4);
-			for (int i = 0; i < numKeyGroups; i++) {
-				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
-			}
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query4)))
-					.thenReturn(Futures.successful(location));
-
-			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
-					.thenReturn(connectException);
-
-			result = client.getKvState(
-					jobId,
-					query4,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected ConnectException exception");
-			} catch (ConnectException ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query4));
-
-			//
-			// Other Exceptions don't lead to a retry no retry
-			//
-			String query5 = "universe";
-			Future<KvStateLocation> exception = Futures.failed(new RuntimeException("Test exception"));
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query5)))
-					.thenReturn(exception);
-
-			client.getKvState(
-					jobId,
-					query5,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId), eq(query5));
-		} finally {
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests queries against multiple servers.
-	 *
-	 * <p>The servers are populated with different keys and the client queries
-	 * all available keys from all servers.
-	 */
-	@Test
-	public void testIntegrationWithKvStateServer() throws Exception {
-		// Config
-		int numServers = 2;
-		int numKeys = 1024;
-		int numKeyGroups = 1;
-
-		JobID jobId = new JobID();
-		JobVertexID jobVertexId = new JobVertexID();
-
-		KvStateServer[] servers = new KvStateServer[numServers];
-		AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
-
-		QueryableStateClient client = null;
-		KvStateClient networkClient = null;
-		AtomicKvStateRequestStats networkClientStats = new AtomicKvStateRequestStats();
-
-		MemoryStateBackend backend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-
-		AbstractKeyedStateBackend<Integer> keyedStateBackend = backend.createKeyedStateBackend(dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
-
-		try {
-			KvStateRegistry[] registries = new KvStateRegistry[numServers];
-			KvStateID[] kvStateIds = new KvStateID[numServers];
-			List<HeapValueState<Integer, VoidNamespace, Integer>> kvStates = new ArrayList<>();
-
-			// Start the servers
-			for (int i = 0; i < numServers; i++) {
-				registries[i] = new KvStateRegistry();
-				serverStats[i] = new AtomicKvStateRequestStats();
-				servers[i] = new KvStateServerImpl(InetAddress.getLocalHost(), 0, 1, 1, registries[i], serverStats[i]);
-				servers[i].start();
-				ValueStateDescriptor<Integer> descriptor =
-						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-
-				RegisteredKeyedBackendStateMetaInfo<VoidNamespace, Integer> registeredKeyedBackendStateMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
-						descriptor.getType(),
-						descriptor.getName(),
-						VoidNamespaceSerializer.INSTANCE,
-						IntSerializer.INSTANCE);
-
-				// Register state
-				HeapValueState<Integer, VoidNamespace, Integer> kvState = new HeapValueState<>(
-						descriptor,
-						new NestedMapsStateTable<>(keyedStateBackend, registeredKeyedBackendStateMetaInfo),
-						IntSerializer.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE);
-
-				kvStates.add(kvState);
-
-				kvStateIds[i] = registries[i].registerKvState(
-						jobId,
-						new JobVertexID(),
-						new KeyGroupRange(i, i),
-						"choco",
-						kvState);
-			}
-
-			int[] expectedRequests = new int[numServers];
-
-			for (int key = 0; key < numKeys; key++) {
-				int targetKeyGroupIndex = MathUtils.murmurHash(key) % numServers;
-				expectedRequests[targetKeyGroupIndex]++;
-
-				HeapValueState<Integer, VoidNamespace, Integer> kvState = kvStates.get(targetKeyGroupIndex);
-
-				keyedStateBackend.setCurrentKey(key);
-				kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
-				kvState.update(1337 + key);
-			}
-
-			// Location lookup service
-			KvStateLocation location = new KvStateLocation(jobId, jobVertexId, numServers, "choco");
-			for (int keyGroupIndex = 0; keyGroupIndex < numServers; keyGroupIndex++) {
-				location.registerKvState(new KeyGroupRange(keyGroupIndex, keyGroupIndex), kvStateIds[keyGroupIndex], servers[keyGroupIndex].getAddress());
-			}
-
-			KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq("choco")))
-					.thenReturn(Futures.successful(location));
-
-			// The client
-			networkClient = new KvStateClient(1, networkClientStats);
-
-			client = new QueryableStateClient(lookupService, networkClient, testActorSystem.dispatcher());
-
-			// Send all queries
-			List<Future<Integer>> futures = new ArrayList<>(numKeys);
-			for (int key = 0; key < numKeys; key++) {
-				ValueStateDescriptor<Integer> descriptor =
-						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-				futures.add(client.getKvState(
-						jobId,
-						"choco",
-						key,
-						BasicTypeInfo.INT_TYPE_INFO,
-						descriptor));
-			}
-
-			// Verify results
-			Future<Iterable<Integer>> future = Futures.sequence(futures, testActorSystem.dispatcher());
-			Iterable<Integer> results = Await.result(future, timeout);
-
-			int index = 0;
-			for (int buffer : results) {
-				assertEquals(1337 + index, buffer);
-				index++;
-			}
-
-			// Verify requests
-			for (int i = 0; i < numServers; i++) {
-				int numRetries = 10;
-				for (int retry = 0; retry < numRetries; retry++) {
-					try {
-						assertEquals("Unexpected number of requests", expectedRequests[i], serverStats[i].getNumRequests());
-						assertEquals("Unexpected success requests", expectedRequests[i], serverStats[i].getNumSuccessful());
-						assertEquals("Unexpected failed requests", 0, serverStats[i].getNumFailed());
-						break;
-					} catch (Throwable t) {
-						// Retry
-						if (retry == numRetries - 1) {
-							throw t;
-						} else {
-							Thread.sleep(100);
-						}
-					}
-				}
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (networkClient != null) {
-				networkClient.shutDown();
-			}
-
-			for (KvStateServer server : servers) {
-				if (server != null) {
-					server.shutDown();
-				}
-			}
-		}
-	}
-
-	/**
-	 * Tests that the QueryableState client correctly caches location lookups
-	 * keyed by both job and name. This test is mainly due to a previous bug due
-	 * to which cache entries were by name only. This is a problem, because the
-	 * same client can be used to query multiple jobs.
-	 */
-	@Test
-	public void testLookupMultipleJobIds() throws Exception {
-		String name = "unique-per-job";
-
-		// Exact contents don't matter here
-		KvStateLocation location = new KvStateLocation(new JobID(), new JobVertexID(), 1, name);
-		location.registerKvState(new KeyGroupRange(0, 0), new KvStateID(), new KvStateServerAddress(InetAddress.getLocalHost(), 892));
-
-		JobID jobId1 = new JobID();
-		JobID jobId2 = new JobID();
-
-		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-
-		when(lookupService.getKvStateLookupInfo(any(JobID.class), anyString()))
-				.thenReturn(Futures.successful(location));
-
-		KvStateClient networkClient = mock(KvStateClient.class);
-		when(networkClient.getKvState(any(KvStateServerAddress.class), any(KvStateID.class), any(byte[].class)))
-				.thenReturn(Futures.successful(new byte[0]));
-
-		QueryableStateClient client = new QueryableStateClient(
-				lookupService,
-				networkClient,
-				testActorSystem.dispatcher());
-
-		ValueStateDescriptor<Integer> stateDesc = new ValueStateDescriptor<>("test", IntSerializer.INSTANCE);
-
-		// Query ies with same name, but different job IDs should lead to a
-		// single lookup per query and job ID.
-		client.getKvState(jobId1, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
-		client.getKvState(jobId2, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
-
-		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId1), eq(name));
-		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId2), eq(name));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
index b982c8e..50ef543 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
@@ -445,4 +445,20 @@ public class FutureUtils {
 
 		return result;
 	}
+
+	// ------------------------------------------------------------------------
+	//  Future Completed with an exception.
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Returns a {@link CompletableFuture} that has failed with the exception
+	 * provided as argument.
+	 * @param throwable the exception to fail the future with.
+	 * @return The failed future.
+	 */
+	public static <T> CompletableFuture<T> getFailedFuture(Throwable throwable) {
+		CompletableFuture<T> failedAttempt = new CompletableFuture<>();
+		failedAttempt.completeExceptionally(throwable);
+		return failedAttempt;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/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 53503ce..d6c5d75 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
@@ -29,14 +29,16 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.query.KvStateClientProxy;
 import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
 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.ExceptionUtils;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,6 +67,9 @@ public class NetworkEnvironment {
 	/** Server for {@link InternalKvState} requests. */
 	private final KvStateServer kvStateServer;
 
+	/** Proxy for the queryable state client. */
+	private final KvStateClientProxy kvStateProxy;
+
 	/** Registry for {@link InternalKvState} instances. */
 	private final KvStateRegistry kvStateRegistry;
 
@@ -76,6 +81,7 @@ public class NetworkEnvironment {
 
 	/** Number of network buffers to use for each outgoing/incoming channel (subpartition/input channel). */
 	private final int networkBuffersPerChannel;
+
 	/** Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate). */
 	private final int extraNetworkBuffersPerGate;
 
@@ -88,6 +94,7 @@ public class NetworkEnvironment {
 			TaskEventDispatcher taskEventDispatcher,
 			KvStateRegistry kvStateRegistry,
 			KvStateServer kvStateServer,
+			KvStateClientProxy kvStateClientProxy,
 			IOMode defaultIOMode,
 			int partitionRequestInitialBackoff,
 			int partitionRequestMaxBackoff,
@@ -101,6 +108,7 @@ public class NetworkEnvironment {
 		this.kvStateRegistry = checkNotNull(kvStateRegistry);
 
 		this.kvStateServer = kvStateServer;
+		this.kvStateProxy = kvStateClientProxy;
 
 		this.defaultIOMode = defaultIOMode;
 
@@ -152,6 +160,10 @@ public class NetworkEnvironment {
 		return kvStateServer;
 	}
 
+	public KvStateClientProxy getKvStateProxy() {
+		return kvStateProxy;
+	}
+
 	public TaskKvStateRegistry createKvStateTaskRegistry(JobID jobId, JobVertexID jobVertexId) {
 		return kvStateRegistry.createTaskRegistry(jobId, jobVertexId);
 	}
@@ -291,17 +303,25 @@ public class NetworkEnvironment {
 			try {
 				LOG.debug("Starting network connection manager");
 				connectionManager.start(resultPartitionManager, taskEventDispatcher);
-			}
-			catch (IOException t) {
+			} catch (IOException t) {
 				throw new IOException("Failed to instantiate network connection manager.", t);
 			}
 
 			if (kvStateServer != null) {
 				try {
-					LOG.debug("Starting the KvState server.");
 					kvStateServer.start();
+					LOG.info("Started Queryable State Data Server @ {}", kvStateServer.getServerAddress());
+				} catch (InterruptedException ie) {
+					throw new IOException("Failed to start the Queryable State Data Server.", ie);
+				}
+			}
+
+			if (kvStateProxy != null) {
+				try {
+					kvStateProxy.start();
+					LOG.info("Started the Queryable State Client Proxy @ {}", kvStateProxy.getServerAddress());
 				} catch (InterruptedException ie) {
-					throw new IOException("Failed to start the KvState server.", ie);
+					throw new IOException("Failed to start the Queryable State Client Proxy.", ie);
 				}
 			}
 		}
@@ -318,11 +338,21 @@ public class NetworkEnvironment {
 
 			LOG.info("Shutting down the network environment and its components.");
 
+			if (kvStateProxy != null) {
+				try {
+					LOG.debug("Shutting down Queryable State Client Proxy.");
+					kvStateProxy.shutdown();
+				} catch (Throwable t) {
+					LOG.warn("Cannot shut down Queryable State Client Proxy.", t);
+				}
+			}
+
 			if (kvStateServer != null) {
 				try {
-					kvStateServer.shutDown();
+					LOG.debug("Shutting down Queryable State Data Server.");
+					kvStateServer.shutdown();
 				} catch (Throwable t) {
-					LOG.warn("Cannot shut down KvState server.", t);
+					LOG.warn("Cannot shut down Queryable State Data Server.", t);
 				}
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateClientProxy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateClientProxy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateClientProxy.java
new file mode 100644
index 0000000..d605952
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateClientProxy.java
@@ -0,0 +1,65 @@
+/*
+ * 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.query;
+
+import org.apache.flink.runtime.instance.ActorGateway;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * An interface for the Queryable State Client Proxy running on each Task Manager in the cluster.
+ *
+ * <p>This proxy is where the Queryable State Client (potentially running outside your Flink
+ * cluster) connects to, and his responsibility is to forward the client's requests to the rest
+ * of the entities participating in fetching the requested state, and running within the cluster.
+ *
+ * <p>These are:
+ * <ol>
+ *     <li> the {@link org.apache.flink.runtime.jobmanager.JobManager Job Manager},
+ *     which is responsible for sending the
+ *     {@link org.apache.flink.runtime.taskmanager.TaskManager Task Manager} storing
+ *     the requested state, and </li>
+ *     <li> the Task Manager having the state itself.</li>
+ * </ol>
+ */
+public interface KvStateClientProxy extends KvStateServer {
+
+	/**
+	 * Updates the active {@link org.apache.flink.runtime.jobmanager.JobManager Job Manager}
+	 * in case of change.
+	 *
+	 * <p>This is useful in settings where high-availability is enabled and
+	 * a failed Job Manager is replaced by a new one.
+	 *
+	 * <p><b>IMPORTANT: </b> this method may be called by a different thread than the {@link #getJobManagerFuture()}.
+	 *
+	 * @param leadingJobManager the currently leading job manager.
+	 * */
+	void updateJobManager(CompletableFuture<ActorGateway> leadingJobManager) throws Exception;
+
+	/**
+	 * Retrieves a future containing the currently leading Job Manager.
+	 *
+	 * <p><b>IMPORTANT: </b> this method may be called by a different thread than the
+	 * {@link #updateJobManager(CompletableFuture)}.
+	 *
+	 * @return A {@link CompletableFuture} containing the currently active Job Manager.
+	 */
+	CompletableFuture<ActorGateway> getJobManagerFuture();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/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 8a213bb..03e8238 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
@@ -31,7 +31,7 @@ import java.util.Arrays;
  * Location information for all key groups of a {@link InternalKvState} instance.
  *
  * <p>This is populated by the {@link KvStateLocationRegistry} and used by the
- * Queryable State Client to target queries.
+ * queryable state to target queries.
  */
 public class KvStateLocation implements Serializable {
 
@@ -183,10 +183,6 @@ public class KvStateLocation implements Serializable {
 		}
 	}
 
-	public static long getSerialVersionUID() {
-		return serialVersionUID;
-	}
-
 	/**
 	 * Registers a KvState instance for the given key group index.
 	 *
@@ -194,7 +190,7 @@ public class KvStateLocation implements Serializable {
 	 * @throws IndexOutOfBoundsException If key group range start < 0 or key group range end >= Number of key groups
 	 * @throws IllegalArgumentException  If no location information registered for a key group index in the range.
 	 */
-	public void unregisterKvState(KeyGroupRange keyGroupRange) {
+	void unregisterKvState(KeyGroupRange keyGroupRange) {
 		if (keyGroupRange.getStartKeyGroup() < 0 || keyGroupRange.getEndKeyGroup() >= numKeyGroups) {
 			throw new IndexOutOfBoundsException("Key group index");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
index 9b14c49..81727fc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
@@ -20,24 +20,21 @@ package org.apache.flink.runtime.query;
 
 /**
  * An interface for the Queryable State Server running on each Task Manager in the cluster.
- * This server is responsible for serving requests coming from the Queryable State Client and
- * requesting <b>locally</b> stored state.
+ * This server is responsible for serving requests coming from the {@link KvStateClientProxy
+ * Queryable State Proxy} and requesting <b>locally</b> stored state.
  */
 public interface KvStateServer {
 
 	/**
-	 * Returns the address of this server.
-	 *
-	 * @return Server address
+	 * Returns the {@link KvStateServerAddress address} the server is listening to.
+	 * @return Server address.
 	 */
-	KvStateServerAddress getAddress();
+	KvStateServerAddress getServerAddress();
 
 
-	/** Starts the proxy. */
+	/** Starts the server. */
 	void start() throws InterruptedException;
 
-	/**
-	 * Shuts down the server and all related thread pools.
-	 */
-	void shutDown();
+	/** Shuts down the server and all related thread pools. */
+	void shutdown();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
index 852d394..8f66734 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java
@@ -36,8 +36,56 @@ public final class QueryableStateUtils {
 	private static final Logger LOG = LoggerFactory.getLogger(QueryableStateUtils.class);
 
 	/**
+	 * Initializes the {@link KvStateClientProxy client proxy} responsible for
+	 * receiving requests from the external (to the cluster) client and forwarding them internally.
+	 *
+	 * @param address the address to bind to.
+	 * @param port the port to listen to.
+	 * @param eventLoopThreads the number of threads to be used to process incoming requests.
+	 * @param queryThreads the number of threads to be used to send the actual state.
+	 * @param stats statistics to be gathered about the incoming requests.
+	 * @return the {@link KvStateClientProxy client proxy}.
+	 */
+	public static KvStateClientProxy createKvStateClientProxy(
+			final InetAddress address,
+			final int port,
+			final int eventLoopThreads,
+			final int queryThreads,
+			final KvStateRequestStats stats) {
+
+		Preconditions.checkNotNull(address, "address");
+		Preconditions.checkNotNull(stats, "stats");
+
+		Preconditions.checkArgument(eventLoopThreads >= 1);
+		Preconditions.checkArgument(queryThreads >= 1);
+
+		try {
+			String classname = "org.apache.flink.queryablestate.client.proxy.KvStateClientProxyImpl";
+			Class<? extends KvStateClientProxy> clazz = Class.forName(classname).asSubclass(KvStateClientProxy.class);
+			Constructor<? extends KvStateClientProxy> constructor = clazz.getConstructor(
+					InetAddress.class,
+					Integer.class,
+					Integer.class,
+					Integer.class,
+					KvStateRequestStats.class);
+			return constructor.newInstance(address, port, eventLoopThreads, queryThreads, stats);
+		} catch (ClassNotFoundException e) {
+			LOG.warn("Could not load Queryable State Client Proxy. " +
+					"Probable reason: flink-queryable-state is not in the classpath");
+			LOG.debug("Caught exception", e);
+			return null;
+		} catch (InvocationTargetException e) {
+			LOG.error("Queryable State Client Proxy could not be created: ", e.getTargetException());
+			return null;
+		} catch (Throwable t) {
+			LOG.error("Failed to instantiate the Queryable State Client Proxy.", t);
+			return null;
+		}
+	}
+
+	/**
 	 * Initializes the {@link KvStateServer server} responsible for sending the
-	 * requested internal state to the Queryable State Client.
+	 * requested internal state to the {@link KvStateClientProxy client proxy}.
 	 *
 	 * @param address the address to bind to.
 	 * @param port the port to listen to.
@@ -74,12 +122,12 @@ public final class QueryableStateUtils {
 					KvStateRequestStats.class);
 			return constructor.newInstance(address, port, eventLoopThreads, queryThreads, kvStateRegistry, stats);
 		} catch (ClassNotFoundException e) {
-			LOG.info("Could not load Queryable State Server. " +
+			LOG.warn("Could not load Queryable State Server. " +
 					"Probable reason: flink-queryable-state is not in the classpath");
 			LOG.debug("Caught exception", e);
 			return null;
 		} catch (InvocationTargetException e) {
-			LOG.error("Queryable State Server could not be created", e.getTargetException());
+			LOG.error("Queryable State Server could not be created: ", e.getTargetException());
 			return null;
 		} catch (Throwable t) {
 			LOG.error("Failed to instantiate the Queryable State Server.", t);

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
index 9781e23..19caf92 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateRequestStats.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.runtime.query.netty;
 
-import org.apache.flink.runtime.query.KvStateServer;
-
 /**
- * Simple statistics for {@link KvStateServer} monitoring.
+ * Simple statistics for
+ * {@link org.apache.flink.runtime.query.KvStateServer} and
+ * {@link org.apache.flink.runtime.query.KvStateClientProxy} monitoring.
  */
 public interface KvStateRequestStats {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
index 37d28de..fed5fc0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
@@ -82,6 +82,7 @@ public class QueryableStateConfiguration {
 	public String toString() {
 		return "QueryableStateConfiguration {" +
 				"enabled=" + enabled +
+				", port=" + port +
 				", numServerThreads=" + numServerThreads +
 				", numQueryThreads=" + numQueryThreads +
 				'}';

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
index 7c5c830..cbf0d95 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
@@ -38,10 +38,11 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
+import org.apache.flink.runtime.query.KvStateClientProxy;
 import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
 import org.apache.flink.runtime.query.QueryableStateUtils;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.KvStateServer;
 import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskexecutor.slot.TimerService;
 import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;
@@ -66,7 +67,7 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 public class TaskManagerServices {
 	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServices.class);
 
-	/** TaskManager services */
+	/** TaskManager services. */
 	private final TaskManagerLocation taskManagerLocation;
 	private final MemoryManager memoryManager;
 	private final IOManager ioManager;
@@ -356,6 +357,7 @@ public class TaskManagerServices {
 		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
 
 		KvStateRegistry kvStateRegistry = new KvStateRegistry();
+		KvStateClientProxy kvClientProxy = null;
 		KvStateServer kvStateServer = null;
 
 		if (taskManagerServicesConfiguration.getQueryableStateConfig().enabled()) {
@@ -367,11 +369,18 @@ public class TaskManagerServices {
 			int numQueryThreads = qsConfig.numQueryThreads() == 0 ?
 					taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numQueryThreads();
 
-			kvStateServer = QueryableStateUtils.createKvStateServer(
+			kvClientProxy = QueryableStateUtils.createKvStateClientProxy(
 					taskManagerServicesConfiguration.getTaskManagerAddress(),
 					qsConfig.port(),
 					numNetworkThreads,
 					numQueryThreads,
+					new DisabledKvStateRequestStats());
+
+			kvStateServer = QueryableStateUtils.createKvStateServer(
+					taskManagerServicesConfiguration.getTaskManagerAddress(),
+					0,
+					numNetworkThreads,
+					numQueryThreads,
 					kvStateRegistry,
 					new DisabledKvStateRequestStats());
 		}
@@ -384,6 +393,7 @@ public class TaskManagerServices {
 			taskEventDispatcher,
 			kvStateRegistry,
 			kvStateServer,
+			kvClientProxy,
 			networkEnvironmentConfiguration.ioMode(),
 			networkEnvironmentConfiguration.partitionRequestInitialBackoff(),
 			networkEnvironmentConfiguration.partitionRequestMaxBackoff(),

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
index f1f7d39..e6643b7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
@@ -189,8 +189,8 @@ public class TaskManagerServicesConfiguration {
 			remoteAddress,
 			slots);
 
-		final QueryableStateConfiguration queryableStateConfig = localCommunication ?
-				QueryableStateConfiguration.disabled() :
+		// @Ufuk todo why was it like this before ???
+		final QueryableStateConfiguration queryableStateConfig =
 				parseQueryableStateConfiguration(configuration);
 
 		// extract memory settings

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 558388c..c370725 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.blob.{BlobClient, BlobService, BlobCacheService}
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager
 import org.apache.flink.runtime.clusterframework.messages.StopCluster
 import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.concurrent.Executors
+import org.apache.flink.runtime.concurrent.{Executors, FutureUtils}
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor
 import org.apache.flink.runtime.execution.ExecutionState
 import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager}
@@ -47,7 +47,7 @@ import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, PartitionInf
 import org.apache.flink.runtime.filecache.FileCache
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution
 import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils}
-import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID}
+import org.apache.flink.runtime.instance.{ActorGateway, AkkaActorGateway, HardwareDescription, InstanceID}
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker
@@ -951,7 +951,7 @@ class TaskManager(
       kvStateRegistry.registerListener(
         new ActorGatewayKvStateRegistryListener(
           jobManagerGateway,
-          kvStateServer.getAddress))
+          kvStateServer.getServerAddress))
     }
 
     // start a blob service, if a blob server is specified
@@ -1423,6 +1423,28 @@ class TaskManager(
   }
 
   override def notifyLeaderAddress(leaderAddress: String, leaderSessionID: UUID): Unit = {
+    val proxy = network.getKvStateProxy
+    if (proxy != null) {
+
+      val askTimeoutString = config.getConfiguration.getString(AkkaOptions.ASK_TIMEOUT)
+
+      val timeout = Duration(askTimeoutString)
+
+      if (!timeout.isFinite) {
+        throw new IllegalConfigurationException(AkkaOptions.ASK_TIMEOUT.key +
+          " is not a finite timeout ('" + askTimeoutString + "')")
+      }
+
+      if (leaderAddress != null) {
+        val actorGwFuture: Future[ActorGateway] =
+          AkkaUtils.getActorRefFuture(
+            leaderAddress, context.system, timeout.asInstanceOf[FiniteDuration]
+          ).map(actor => new AkkaActorGateway(actor, leaderSessionID))(context.system.dispatcher)
+
+        proxy.updateJobManager(FutureUtils.toJava(actorGwFuture))
+      }
+    }
+
     self ! JobManagerLeaderAddress(leaderAddress, leaderSessionID)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index 826ae3f..ef2d5c2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -69,6 +69,7 @@ public class NetworkEnvironmentTest {
 			new TaskEventDispatcher(),
 			new KvStateRegistry(),
 			null,
+			null,
 			IOManager.IOMode.SYNC,
 			0,
 			0,

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 052699a..6dabcd3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -145,6 +145,7 @@ public class TaskManagerComponentsStartupShutdownTest extends TestLogger {
 				new TaskEventDispatcher(),
 				new KvStateRegistry(),
 				null,
+				null,
 				netConf.ioMode(),
 				netConf.partitionRequestInitialBackoff(),
 				netConf.partitionRequestMaxBackoff(),


[05/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
new file mode 100644
index 0000000..c37c822
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
@@ -0,0 +1,728 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.queryablestate.UnknownKvStateID;
+import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
+import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.server.ChunkedByteBuf;
+import org.apache.flink.queryablestate.server.KvStateServerHandler;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateRegistryListener;
+import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link KvStateServerHandler}.
+ */
+public class KvStateServerHandlerTest extends TestLogger {
+
+	/** Shared Thread pool for query execution. */
+	private static final ExecutorService TEST_THREAD_POOL = Executors.newSingleThreadExecutor();
+
+	private static final int READ_TIMEOUT_MILLIS = 10000;
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (TEST_THREAD_POOL != null) {
+			TEST_THREAD_POOL.shutdown();
+		}
+	}
+
+	/**
+	 * Tests a simple successful query via an EmbeddedChannel.
+	 */
+	@Test
+	public void testSimpleQuery() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		// Register state
+		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+		desc.setQueryable("vanilla");
+
+		int numKeyGroups = 1;
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(registry);
+		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
+
+		final TestRegistryListener registryListener = new TestRegistryListener();
+		registry.registerListener(registryListener);
+
+		// Update the KvState and request it
+		int expectedValue = 712828289;
+
+		int key = 99812822;
+		backend.setCurrentKey(key);
+		ValueState<Integer> state = backend.getPartitionedState(
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE,
+				desc);
+
+		state.update(expectedValue);
+
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				key,
+				IntSerializer.INSTANCE,
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE);
+
+		long requestId = Integer.MAX_VALUE + 182828L;
+
+		assertTrue(registryListener.registrationName.equals("vanilla"));
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				requestId,
+				registryListener.kvStateId,
+				serializedKeyAndNamespace);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestResult response = MessageSerializer.deserializeKvStateRequestResult(buf);
+
+		assertEquals(requestId, response.getRequestId());
+
+		int actualValue = KvStateSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
+		assertEquals(expectedValue, actualValue);
+
+		assertEquals(stats.toString(), 1, stats.getNumRequests());
+
+		// Wait for async successful request report
+		long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
+		while (stats.getNumSuccessful() != 1 && System.nanoTime() <= deadline) {
+			Thread.sleep(10);
+		}
+
+		assertEquals(stats.toString(), 1, stats.getNumSuccessful());
+	}
+
+	/**
+	 * Tests the failure response with {@link UnknownKvStateID} as cause on
+	 * queries for unregistered KvStateIDs.
+	 */
+	@Test
+	public void testQueryUnknownKvStateID() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		long requestId = Integer.MAX_VALUE + 182828L;
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				requestId,
+				new KvStateID(),
+				new byte[0]);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+
+		assertEquals(requestId, response.getRequestId());
+
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateID);
+
+		assertEquals(1, stats.getNumRequests());
+		assertEquals(1, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests the failure response with {@link UnknownKeyOrNamespace} as cause
+	 * on queries for non-existing keys.
+	 */
+	@Test
+	public void testQueryUnknownKey() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		int numKeyGroups = 1;
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(registry);
+		KeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
+
+		final TestRegistryListener registryListener = new TestRegistryListener();
+		registry.registerListener(registryListener);
+
+		// Register state
+		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+		desc.setQueryable("vanilla");
+
+		backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);
+
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				1238283,
+				IntSerializer.INSTANCE,
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE);
+
+		long requestId = Integer.MAX_VALUE + 22982L;
+
+		assertTrue(registryListener.registrationName.equals("vanilla"));
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				requestId,
+				registryListener.kvStateId,
+				serializedKeyAndNamespace);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+
+		assertEquals(requestId, response.getRequestId());
+
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespace);
+
+		assertEquals(1, stats.getNumRequests());
+		assertEquals(1, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests the failure response on a failure on the {@link InternalKvState#getSerializedValue(byte[])}
+	 * call.
+	 */
+	@Test
+	public void testFailureOnGetSerializedValue() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		// Failing KvState
+		InternalKvState<?> kvState = mock(InternalKvState.class);
+		when(kvState.getSerializedValue(any(byte[].class)))
+				.thenThrow(new RuntimeException("Expected test Exception"));
+
+		KvStateID kvStateId = registry.registerKvState(
+				new JobID(),
+				new JobVertexID(),
+				new KeyGroupRange(0, 0),
+				"vanilla",
+				kvState);
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				282872,
+				kvStateId,
+				new byte[0]);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+
+		assertTrue(response.getCause().getMessage().contains("Expected test Exception"));
+
+		assertEquals(1, stats.getNumRequests());
+		assertEquals(1, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests that the channel is closed if an Exception reaches the channel
+	 * handler.
+	 */
+	@Test
+	public void testCloseChannelOnExceptionCaught() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(handler);
+
+		channel.pipeline().fireExceptionCaught(new RuntimeException("Expected test Exception"));
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
+		Throwable response = MessageSerializer.deserializeServerFailure(buf);
+
+		assertTrue(response.getMessage().contains("Expected test Exception"));
+
+		channel.closeFuture().await(READ_TIMEOUT_MILLIS);
+		assertFalse(channel.isActive());
+	}
+
+	/**
+	 * Tests the failure response on a rejected execution, because the query
+	 * executor has been closed.
+	 */
+	@Test
+	public void testQueryExecutorShutDown() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		ExecutorService closedExecutor = Executors.newSingleThreadExecutor();
+		closedExecutor.shutdown();
+		assertTrue(closedExecutor.isShutdown());
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, closedExecutor, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		int numKeyGroups = 1;
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(registry);
+		KeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
+
+		final TestRegistryListener registryListener = new TestRegistryListener();
+		registry.registerListener(registryListener);
+
+		// Register state
+		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+		desc.setQueryable("vanilla");
+
+		backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);
+
+		assertTrue(registryListener.registrationName.equals("vanilla"));
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				282872,
+				registryListener.kvStateId,
+				new byte[0]);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+
+		assertTrue(response.getCause().getMessage().contains("RejectedExecutionException"));
+
+		assertEquals(1, stats.getNumRequests());
+		assertEquals(1, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests response on unexpected messages.
+	 */
+	@Test
+	public void testUnexpectedMessage() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		// Write the request and wait for the response
+		ByteBuf unexpectedMessage = Unpooled.buffer(8);
+		unexpectedMessage.writeInt(4);
+		unexpectedMessage.writeInt(123238213);
+
+		channel.writeInbound(unexpectedMessage);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
+		Throwable response = MessageSerializer.deserializeServerFailure(buf);
+
+		assertEquals(0, stats.getNumRequests());
+		assertEquals(0, stats.getNumFailed());
+
+		unexpectedMessage = MessageSerializer.serializeKvStateRequestResult(
+				channel.alloc(),
+				192,
+				new byte[0]);
+
+		channel.writeInbound(unexpectedMessage);
+
+		buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
+		response = MessageSerializer.deserializeServerFailure(buf);
+
+		assertTrue("Unexpected failure cause " + response.getClass().getName(), response instanceof IllegalArgumentException);
+
+		assertEquals(0, stats.getNumRequests());
+		assertEquals(0, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests that incoming buffer instances are recycled.
+	 */
+	@Test
+	public void testIncomingBufferIsRecycled() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				282872,
+				new KvStateID(),
+				new byte[0]);
+
+		assertEquals(1, request.refCnt());
+
+		// Write regular request
+		channel.writeInbound(request);
+		assertEquals("Buffer not recycled", 0, request.refCnt());
+
+		// Write unexpected msg
+		ByteBuf unexpected = channel.alloc().buffer(8);
+		unexpected.writeInt(4);
+		unexpected.writeInt(4);
+
+		assertEquals(1, unexpected.refCnt());
+
+		channel.writeInbound(unexpected);
+		assertEquals("Buffer not recycled", 0, unexpected.refCnt());
+	}
+
+	/**
+	 * Tests the failure response if the serializers don't match.
+	 */
+	@Test
+	public void testSerializerMismatch() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		int numKeyGroups = 1;
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(registry);
+		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
+
+		final TestRegistryListener registryListener = new TestRegistryListener();
+		registry.registerListener(registryListener);
+
+		// Register state
+		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+		desc.setQueryable("vanilla");
+
+		ValueState<Integer> state = backend.getPartitionedState(
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE,
+				desc);
+
+		int key = 99812822;
+
+		// Update the KvState
+		backend.setCurrentKey(key);
+		state.update(712828289);
+
+		byte[] wrongKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				"wrong-key-type",
+				StringSerializer.INSTANCE,
+				"wrong-namespace-type",
+				StringSerializer.INSTANCE);
+
+		byte[] wrongNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				key,
+				IntSerializer.INSTANCE,
+				"wrong-namespace-type",
+				StringSerializer.INSTANCE);
+
+		assertTrue(registryListener.registrationName.equals("vanilla"));
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				182828,
+				registryListener.kvStateId,
+				wrongKeyAndNamespace);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		assertEquals(182828, response.getRequestId());
+		assertTrue(response.getCause().getMessage().contains("IOException"));
+
+		// Repeat with wrong namespace only
+		request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				182829,
+				registryListener.kvStateId,
+				wrongNamespace);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		buf = (ByteBuf) readInboundBlocking(channel);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify the response
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		assertEquals(182829, response.getRequestId());
+		assertTrue(response.getCause().getMessage().contains("IOException"));
+
+		assertEquals(2, stats.getNumRequests());
+		assertEquals(2, stats.getNumFailed());
+	}
+
+	/**
+	 * Tests that large responses are chunked.
+	 */
+	@Test
+	public void testChunkedResponse() throws Exception {
+		KvStateRegistry registry = new KvStateRegistry();
+		KvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
+
+		int numKeyGroups = 1;
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(registry);
+		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
+
+		final TestRegistryListener registryListener = new TestRegistryListener();
+		registry.registerListener(registryListener);
+
+		// Register state
+		ValueStateDescriptor<byte[]> desc = new ValueStateDescriptor<>("any", BytePrimitiveArraySerializer.INSTANCE);
+		desc.setQueryable("vanilla");
+
+		ValueState<byte[]> state = backend.getPartitionedState(
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE,
+				desc);
+
+		// Update KvState
+		byte[] bytes = new byte[2 * channel.config().getWriteBufferHighWaterMark()];
+
+		byte current = 0;
+		for (int i = 0; i < bytes.length; i++) {
+			bytes[i] = current++;
+		}
+
+		int key = 99812822;
+		backend.setCurrentKey(key);
+		state.update(bytes);
+
+		// Request
+		byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+				key,
+				IntSerializer.INSTANCE,
+				VoidNamespace.INSTANCE,
+				VoidNamespaceSerializer.INSTANCE);
+
+		long requestId = Integer.MAX_VALUE + 182828L;
+
+		assertTrue(registryListener.registrationName.equals("vanilla"));
+
+		ByteBuf request = MessageSerializer.serializeKvStateRequest(
+				channel.alloc(),
+				requestId,
+				registryListener.kvStateId,
+				serializedKeyAndNamespace);
+
+		// Write the request and wait for the response
+		channel.writeInbound(request);
+
+		Object msg = readInboundBlocking(channel);
+		assertTrue("Not ChunkedByteBuf", msg instanceof ChunkedByteBuf);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Queries the embedded channel for data.
+	 */
+	private Object readInboundBlocking(EmbeddedChannel channel) throws InterruptedException, TimeoutException {
+		final int sleepMillis = 50;
+
+		int sleptMillis = 0;
+
+		Object msg = null;
+		while (sleptMillis < READ_TIMEOUT_MILLIS &&
+				(msg = channel.readOutbound()) == null) {
+
+			Thread.sleep(sleepMillis);
+			sleptMillis += sleepMillis;
+		}
+
+		if (msg == null) {
+			throw new TimeoutException();
+		} else {
+			return msg;
+		}
+	}
+
+	/**
+	 * Frame length decoder (expected by the serialized messages).
+	 */
+	private ChannelHandler getFrameDecoder() {
+		return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4);
+	}
+
+	/**
+	 * A listener that keeps the last updated KvState information so that a test
+	 * can retrieve it.
+	 */
+	static class TestRegistryListener implements KvStateRegistryListener {
+		volatile JobVertexID jobVertexID;
+		volatile KeyGroupRange keyGroupIndex;
+		volatile String registrationName;
+		volatile KvStateID kvStateId;
+
+		@Override
+		public void notifyKvStateRegistered(JobID jobId,
+				JobVertexID jobVertexId,
+				KeyGroupRange keyGroupRange,
+				String registrationName,
+				KvStateID kvStateId) {
+			this.jobVertexID = jobVertexId;
+			this.keyGroupIndex = keyGroupRange;
+			this.registrationName = registrationName;
+			this.kvStateId = kvStateId;
+		}
+
+		@Override
+		public void notifyKvStateUnregistered(JobID jobId,
+				JobVertexID jobVertexId,
+				KeyGroupRange keyGroupRange,
+				String registrationName) {
+
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
new file mode 100644
index 0000000..9332e68
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+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.memory.MemoryStateBackend;
+
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link KvStateServer}.
+ */
+public class KvStateServerTest {
+
+	// Thread pool for client bootstrap (shared between tests)
+	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
+
+	private static final int TIMEOUT_MILLIS = 10000;
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (NIO_GROUP != null) {
+			NIO_GROUP.shutdownGracefully();
+		}
+	}
+
+	/**
+	 * Tests a simple successful query via a SocketChannel.
+	 */
+	@Test
+	public void testSimpleRequest() throws Exception {
+		KvStateServer server = null;
+		Bootstrap bootstrap = null;
+		try {
+			KvStateRegistry registry = new KvStateRegistry();
+			KvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+			server = new KvStateServerImpl(InetAddress.getLocalHost(), 0, 1, 1, registry, stats);
+			server.start();
+
+			KvStateServerAddress serverAddress = server.getAddress();
+			int numKeyGroups = 1;
+			AbstractStateBackend abstractBackend = new MemoryStateBackend();
+			DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+			dummyEnv.setKvStateRegistry(registry);
+			AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+					dummyEnv,
+					new JobID(),
+					"test_op",
+					IntSerializer.INSTANCE,
+					numKeyGroups,
+					new KeyGroupRange(0, 0),
+					registry.createTaskRegistry(new JobID(), new JobVertexID()));
+
+			final KvStateServerHandlerTest.TestRegistryListener registryListener =
+					new KvStateServerHandlerTest.TestRegistryListener();
+
+			registry.registerListener(registryListener);
+
+			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+			desc.setQueryable("vanilla");
+
+			ValueState<Integer> state = backend.getPartitionedState(
+					VoidNamespace.INSTANCE,
+					VoidNamespaceSerializer.INSTANCE,
+					desc);
+
+			// Update KvState
+			int expectedValue = 712828289;
+
+			int key = 99812822;
+			backend.setCurrentKey(key);
+			state.update(expectedValue);
+
+			// Request
+			byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+					key,
+					IntSerializer.INSTANCE,
+					VoidNamespace.INSTANCE,
+					VoidNamespaceSerializer.INSTANCE);
+
+			// Connect to the server
+			final BlockingQueue<ByteBuf> responses = new LinkedBlockingQueue<>();
+			bootstrap = createBootstrap(
+					new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4),
+					new ChannelInboundHandlerAdapter() {
+						@Override
+						public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+							responses.add((ByteBuf) msg);
+						}
+					});
+
+			Channel channel = bootstrap
+					.connect(serverAddress.getHost(), serverAddress.getPort())
+					.sync().channel();
+
+			long requestId = Integer.MAX_VALUE + 182828L;
+
+			assertTrue(registryListener.registrationName.equals("vanilla"));
+			ByteBuf request = MessageSerializer.serializeKvStateRequest(
+					channel.alloc(),
+					requestId,
+					registryListener.kvStateId,
+					serializedKeyAndNamespace);
+
+			channel.writeAndFlush(request);
+
+			ByteBuf buf = responses.poll(TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+
+			assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+			KvStateRequestResult response = MessageSerializer.deserializeKvStateRequestResult(buf);
+
+			assertEquals(requestId, response.getRequestId());
+			int actualValue = KvStateSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
+			assertEquals(expectedValue, actualValue);
+		} finally {
+			if (server != null) {
+				server.shutDown();
+			}
+
+			if (bootstrap != null) {
+				EventLoopGroup group = bootstrap.group();
+				if (group != null) {
+					group.shutdownGracefully();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Creates a client bootstrap.
+	 */
+	private Bootstrap createBootstrap(final ChannelHandler... handlers) {
+		return new Bootstrap().group(NIO_GROUP).channel(NioSocketChannel.class)
+				.handler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					protected void initChannel(SocketChannel ch) throws Exception {
+						ch.pipeline().addLast(handlers);
+					}
+				});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
new file mode 100644
index 0000000..5d4a861
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/QueryableStateClientTest.java
@@ -0,0 +1,458 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.queryablestate.UnknownKvStateID;
+import org.apache.flink.queryablestate.UnknownKvStateKeyGroupLocation;
+import org.apache.flink.queryablestate.client.KvStateClient;
+import org.apache.flink.queryablestate.client.KvStateLocationLookupService;
+import org.apache.flink.queryablestate.client.QueryableStateClient;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
+import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.heap.HeapValueState;
+import org.apache.flink.runtime.state.heap.NestedMapsStateTable;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.MathUtils;
+
+import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for {@link QueryableStateClient}.
+ */
+public class QueryableStateClientTest {
+
+	private static final ActorSystem testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
+
+	private static final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS);
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (testActorSystem != null) {
+			testActorSystem.shutdown();
+		}
+	}
+
+	/**
+	 * All failures should lead to a retry with a forced location lookup.
+	 *
+	 * <p>UnknownKvStateID, UnknownKvStateKeyGroupLocation, UnknownKvStateLocation,
+	 * ConnectException are checked explicitly as these indicate out-of-sync
+	 * KvStateLocation.
+	 */
+	@Test
+	public void testForceLookupOnOutdatedLocation() throws Exception {
+		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
+		KvStateClient networkClient = mock(KvStateClient.class);
+
+		QueryableStateClient client = new QueryableStateClient(
+				lookupService,
+				networkClient,
+				testActorSystem.dispatcher());
+
+		try {
+			JobID jobId = new JobID();
+			int numKeyGroups = 4;
+
+			//
+			// UnknownKvStateLocation
+			//
+			String query1 = "lucky";
+
+			Future<KvStateLocation> unknownKvStateLocation = Futures.failed(
+					new UnknownKvStateLocation(query1));
+
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query1)))
+					.thenReturn(unknownKvStateLocation);
+
+			Future<Integer> result = client.getKvState(
+					jobId,
+					query1,
+					0,
+					BasicTypeInfo.INT_TYPE_INFO,
+					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
+
+			try {
+				Await.result(result, timeout);
+				fail("Did not throw expected UnknownKvStateLocation exception");
+			} catch (UnknownKvStateLocation ignored) {
+				// Expected
+			}
+
+			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query1));
+
+			//
+			// UnknownKvStateKeyGroupLocation
+			//
+			String query2 = "unlucky";
+
+			Future<KvStateLocation> unknownKeyGroupLocation = Futures.successful(
+					new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query2));
+
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query2)))
+					.thenReturn(unknownKeyGroupLocation);
+
+			result = client.getKvState(
+					jobId,
+					query2,
+					0,
+					BasicTypeInfo.INT_TYPE_INFO,
+					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
+
+			try {
+				Await.result(result, timeout);
+				fail("Did not throw expected UnknownKvStateKeyGroupLocation exception");
+			} catch (UnknownKvStateKeyGroupLocation ignored) {
+				// Expected
+			}
+
+			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query2));
+
+			//
+			// UnknownKvStateID
+			//
+			String query3 = "water";
+			KvStateID kvStateId = new KvStateID();
+			Future<byte[]> unknownKvStateId = Futures.failed(new UnknownKvStateID(kvStateId));
+
+			KvStateServerAddress serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 12323);
+			KvStateLocation location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query3);
+			for (int i = 0; i < numKeyGroups; i++) {
+				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
+			}
+
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query3)))
+					.thenReturn(Futures.successful(location));
+
+			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
+					.thenReturn(unknownKvStateId);
+
+			result = client.getKvState(
+					jobId,
+					query3,
+					0,
+					BasicTypeInfo.INT_TYPE_INFO,
+					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
+
+			try {
+				Await.result(result, timeout);
+				fail("Did not throw expected UnknownKvStateID exception");
+			} catch (UnknownKvStateID ignored) {
+				// Expected
+			}
+
+			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query3));
+
+			//
+			// ConnectException
+			//
+			String query4 = "space";
+			Future<byte[]> connectException = Futures.failed(new ConnectException());
+			kvStateId = new KvStateID();
+
+			serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 11123);
+			location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query4);
+			for (int i = 0; i < numKeyGroups; i++) {
+				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
+			}
+
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query4)))
+					.thenReturn(Futures.successful(location));
+
+			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
+					.thenReturn(connectException);
+
+			result = client.getKvState(
+					jobId,
+					query4,
+					0,
+					BasicTypeInfo.INT_TYPE_INFO,
+					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
+
+			try {
+				Await.result(result, timeout);
+				fail("Did not throw expected ConnectException exception");
+			} catch (ConnectException ignored) {
+				// Expected
+			}
+
+			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query4));
+
+			//
+			// Other Exceptions don't lead to a retry no retry
+			//
+			String query5 = "universe";
+			Future<KvStateLocation> exception = Futures.failed(new RuntimeException("Test exception"));
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query5)))
+					.thenReturn(exception);
+
+			client.getKvState(
+					jobId,
+					query5,
+					0,
+					BasicTypeInfo.INT_TYPE_INFO,
+					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
+
+			verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId), eq(query5));
+		} finally {
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Tests queries against multiple servers.
+	 *
+	 * <p>The servers are populated with different keys and the client queries
+	 * all available keys from all servers.
+	 */
+	@Test
+	public void testIntegrationWithKvStateServer() throws Exception {
+		// Config
+		int numServers = 2;
+		int numKeys = 1024;
+		int numKeyGroups = 1;
+
+		JobID jobId = new JobID();
+		JobVertexID jobVertexId = new JobVertexID();
+
+		KvStateServer[] servers = new KvStateServer[numServers];
+		AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
+
+		QueryableStateClient client = null;
+		KvStateClient networkClient = null;
+		AtomicKvStateRequestStats networkClientStats = new AtomicKvStateRequestStats();
+
+		MemoryStateBackend backend = new MemoryStateBackend();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+
+		AbstractKeyedStateBackend<Integer> keyedStateBackend = backend.createKeyedStateBackend(dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
+
+		try {
+			KvStateRegistry[] registries = new KvStateRegistry[numServers];
+			KvStateID[] kvStateIds = new KvStateID[numServers];
+			List<HeapValueState<Integer, VoidNamespace, Integer>> kvStates = new ArrayList<>();
+
+			// Start the servers
+			for (int i = 0; i < numServers; i++) {
+				registries[i] = new KvStateRegistry();
+				serverStats[i] = new AtomicKvStateRequestStats();
+				servers[i] = new KvStateServerImpl(InetAddress.getLocalHost(), 0, 1, 1, registries[i], serverStats[i]);
+				servers[i].start();
+				ValueStateDescriptor<Integer> descriptor =
+						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+
+				RegisteredKeyedBackendStateMetaInfo<VoidNamespace, Integer> registeredKeyedBackendStateMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
+						descriptor.getType(),
+						descriptor.getName(),
+						VoidNamespaceSerializer.INSTANCE,
+						IntSerializer.INSTANCE);
+
+				// Register state
+				HeapValueState<Integer, VoidNamespace, Integer> kvState = new HeapValueState<>(
+						descriptor,
+						new NestedMapsStateTable<>(keyedStateBackend, registeredKeyedBackendStateMetaInfo),
+						IntSerializer.INSTANCE,
+						VoidNamespaceSerializer.INSTANCE);
+
+				kvStates.add(kvState);
+
+				kvStateIds[i] = registries[i].registerKvState(
+						jobId,
+						new JobVertexID(),
+						new KeyGroupRange(i, i),
+						"choco",
+						kvState);
+			}
+
+			int[] expectedRequests = new int[numServers];
+
+			for (int key = 0; key < numKeys; key++) {
+				int targetKeyGroupIndex = MathUtils.murmurHash(key) % numServers;
+				expectedRequests[targetKeyGroupIndex]++;
+
+				HeapValueState<Integer, VoidNamespace, Integer> kvState = kvStates.get(targetKeyGroupIndex);
+
+				keyedStateBackend.setCurrentKey(key);
+				kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
+				kvState.update(1337 + key);
+			}
+
+			// Location lookup service
+			KvStateLocation location = new KvStateLocation(jobId, jobVertexId, numServers, "choco");
+			for (int keyGroupIndex = 0; keyGroupIndex < numServers; keyGroupIndex++) {
+				location.registerKvState(new KeyGroupRange(keyGroupIndex, keyGroupIndex), kvStateIds[keyGroupIndex], servers[keyGroupIndex].getAddress());
+			}
+
+			KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
+			when(lookupService.getKvStateLookupInfo(eq(jobId), eq("choco")))
+					.thenReturn(Futures.successful(location));
+
+			// The client
+			networkClient = new KvStateClient(1, networkClientStats);
+
+			client = new QueryableStateClient(lookupService, networkClient, testActorSystem.dispatcher());
+
+			// Send all queries
+			List<Future<Integer>> futures = new ArrayList<>(numKeys);
+			for (int key = 0; key < numKeys; key++) {
+				ValueStateDescriptor<Integer> descriptor =
+						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+				futures.add(client.getKvState(
+						jobId,
+						"choco",
+						key,
+						BasicTypeInfo.INT_TYPE_INFO,
+						descriptor));
+			}
+
+			// Verify results
+			Future<Iterable<Integer>> future = Futures.sequence(futures, testActorSystem.dispatcher());
+			Iterable<Integer> results = Await.result(future, timeout);
+
+			int index = 0;
+			for (int buffer : results) {
+				assertEquals(1337 + index, buffer);
+				index++;
+			}
+
+			// Verify requests
+			for (int i = 0; i < numServers; i++) {
+				int numRetries = 10;
+				for (int retry = 0; retry < numRetries; retry++) {
+					try {
+						assertEquals("Unexpected number of requests", expectedRequests[i], serverStats[i].getNumRequests());
+						assertEquals("Unexpected success requests", expectedRequests[i], serverStats[i].getNumSuccessful());
+						assertEquals("Unexpected failed requests", 0, serverStats[i].getNumFailed());
+						break;
+					} catch (Throwable t) {
+						// Retry
+						if (retry == numRetries - 1) {
+							throw t;
+						} else {
+							Thread.sleep(100);
+						}
+					}
+				}
+			}
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			if (networkClient != null) {
+				networkClient.shutDown();
+			}
+
+			for (KvStateServer server : servers) {
+				if (server != null) {
+					server.shutDown();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Tests that the QueryableState client correctly caches location lookups
+	 * keyed by both job and name. This test is mainly due to a previous bug due
+	 * to which cache entries were by name only. This is a problem, because the
+	 * same client can be used to query multiple jobs.
+	 */
+	@Test
+	public void testLookupMultipleJobIds() throws Exception {
+		String name = "unique-per-job";
+
+		// Exact contents don't matter here
+		KvStateLocation location = new KvStateLocation(new JobID(), new JobVertexID(), 1, name);
+		location.registerKvState(new KeyGroupRange(0, 0), new KvStateID(), new KvStateServerAddress(InetAddress.getLocalHost(), 892));
+
+		JobID jobId1 = new JobID();
+		JobID jobId2 = new JobID();
+
+		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
+
+		when(lookupService.getKvStateLookupInfo(any(JobID.class), anyString()))
+				.thenReturn(Futures.successful(location));
+
+		KvStateClient networkClient = mock(KvStateClient.class);
+		when(networkClient.getKvState(any(KvStateServerAddress.class), any(KvStateID.class), any(byte[].class)))
+				.thenReturn(Futures.successful(new byte[0]));
+
+		QueryableStateClient client = new QueryableStateClient(
+				lookupService,
+				networkClient,
+				testActorSystem.dispatcher());
+
+		ValueStateDescriptor<Integer> stateDesc = new ValueStateDescriptor<>("test", IntSerializer.INSTANCE);
+
+		// Query ies with same name, but different job IDs should lead to a
+		// single lookup per query and job ID.
+		client.getKvState(jobId1, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
+		client.getKvState(jobId2, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
+
+		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId1), eq(name));
+		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId2), eq(name));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties b/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..10792cd
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/resources/log4j-test.properties
@@ -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.
+#
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+log4j.rootLogger=OFF, testlogger
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR
+log4j.logger.org.apache.zookeeper=OFF

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/pom.xml
----------------------------------------------------------------------
diff --git a/flink-queryable-state/pom.xml b/flink-queryable-state/pom.xml
new file mode 100644
index 0000000..e9e7496
--- /dev/null
+++ b/flink-queryable-state/pom.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-parent</artifactId>
+        <version>1.4-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-queryable-state</artifactId>
+    <name>flink-queryable-state</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>flink-queryable-state-java</module>
+       <!-- <module>flink-state-client-scala</module>-->
+    </modules>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 9193859..53503ce 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
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 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.query.KvStateServer;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.runtime.taskmanager.TaskManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupService.java
deleted file mode 100644
index a37a3ac..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupService.java
+++ /dev/null
@@ -1,322 +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.query;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.util.Preconditions;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-import akka.dispatch.Recover;
-import akka.pattern.Patterns;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.UUID;
-import java.util.concurrent.Callable;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-import scala.reflect.ClassTag$;
-
-/**
- * Akka-based {@link KvStateLocationLookupService} that retrieves the current
- * JobManager address and uses it for lookups.
- */
-class AkkaKvStateLocationLookupService implements KvStateLocationLookupService, LeaderRetrievalListener {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateLocationLookupService.class);
-
-	/** Future returned when no JobManager is available. */
-	private static final Future<ActorGateway> UNKNOWN_JOB_MANAGER = Futures.failed(new UnknownJobManager());
-
-	/** Leader retrieval service to retrieve the current job manager. */
-	private final LeaderRetrievalService leaderRetrievalService;
-
-	/** The actor system used to resolve the JobManager address. */
-	private final ActorSystem actorSystem;
-
-	/** Timeout for JobManager ask-requests. */
-	private final FiniteDuration askTimeout;
-
-	/** Retry strategy factory on future failures. */
-	private final LookupRetryStrategyFactory retryStrategyFactory;
-
-	/** Current job manager future. */
-	private volatile Future<ActorGateway> jobManagerFuture = UNKNOWN_JOB_MANAGER;
-
-	/**
-	 * Creates the Akka-based {@link KvStateLocationLookupService}.
-	 *
-	 * @param leaderRetrievalService Leader retrieval service to use.
-	 * @param actorSystem            Actor system to use.
-	 * @param askTimeout             Timeout for JobManager ask-requests.
-	 * @param retryStrategyFactory   Retry strategy if no JobManager available.
-	 */
-	AkkaKvStateLocationLookupService(
-			LeaderRetrievalService leaderRetrievalService,
-			ActorSystem actorSystem,
-			FiniteDuration askTimeout,
-			LookupRetryStrategyFactory retryStrategyFactory) {
-
-		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService, "Leader retrieval service");
-		this.actorSystem = Preconditions.checkNotNull(actorSystem, "Actor system");
-		this.askTimeout = Preconditions.checkNotNull(askTimeout, "Ask Timeout");
-		this.retryStrategyFactory = Preconditions.checkNotNull(retryStrategyFactory, "Retry strategy factory");
-	}
-
-	public void start() {
-		try {
-			leaderRetrievalService.start(this);
-		} catch (Exception e) {
-			LOG.error("Failed to start leader retrieval service", e);
-			throw new RuntimeException(e);
-		}
-	}
-
-	public void shutDown() {
-		try {
-			leaderRetrievalService.stop();
-		} catch (Exception e) {
-			LOG.error("Failed to stop leader retrieval service", e);
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public Future<KvStateLocation> getKvStateLookupInfo(final JobID jobId, final String registrationName) {
-		return getKvStateLookupInfo(jobId, registrationName, retryStrategyFactory.createRetryStrategy());
-	}
-
-	/**
-	 * Returns a future holding the {@link KvStateLocation} for the given job
-	 * and KvState registration name.
-	 *
-	 * <p>If there is currently no JobManager registered with the service, the
-	 * request is retried. The retry behaviour is specified by the
-	 * {@link LookupRetryStrategy} of the lookup service.
-	 *
-	 * @param jobId               JobID the KvState instance belongs to
-	 * @param registrationName    Name under which the KvState has been registered
-	 * @param lookupRetryStrategy Retry strategy to use for retries on UnknownJobManager failures.
-	 * @return Future holding the {@link KvStateLocation}
-	 */
-	@SuppressWarnings("unchecked")
-	private Future<KvStateLocation> getKvStateLookupInfo(
-			final JobID jobId,
-			final String registrationName,
-			final LookupRetryStrategy lookupRetryStrategy) {
-
-		return jobManagerFuture
-				.flatMap(new Mapper<ActorGateway, Future<Object>>() {
-					@Override
-					public Future<Object> apply(ActorGateway jobManager) {
-						// Lookup the KvStateLocation
-						Object msg = new KvStateMessage.LookupKvStateLocation(jobId, registrationName);
-						return jobManager.ask(msg, askTimeout);
-					}
-				}, actorSystem.dispatcher())
-				.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class))
-				.recoverWith(new Recover<Future<KvStateLocation>>() {
-					@Override
-					public Future<KvStateLocation> recover(Throwable failure) throws Throwable {
-						// If the Future fails with UnknownJobManager, retry
-						// the request. Otherwise all Futures will be failed
-						// during the start up phase, when the JobManager did
-						// not notify this service yet or leadership is lost
-						// intermittently.
-						if (failure instanceof UnknownJobManager && lookupRetryStrategy.tryRetry()) {
-							return Patterns.after(
-									lookupRetryStrategy.getRetryDelay(),
-									actorSystem.scheduler(),
-									actorSystem.dispatcher(),
-									new Callable<Future<KvStateLocation>>() {
-										@Override
-										public Future<KvStateLocation> call() throws Exception {
-											return getKvStateLookupInfo(
-													jobId,
-													registrationName,
-													lookupRetryStrategy);
-										}
-									});
-						} else {
-							return Futures.failed(failure);
-						}
-					}
-				}, actorSystem.dispatcher());
-	}
-
-	@Override
-	public void notifyLeaderAddress(String leaderAddress, final UUID leaderSessionID) {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Received leader address notification {}:{}", leaderAddress, leaderSessionID);
-		}
-
-		if (leaderAddress == null) {
-			jobManagerFuture = UNKNOWN_JOB_MANAGER;
-		} else {
-			jobManagerFuture = AkkaUtils.getActorRefFuture(leaderAddress, actorSystem, askTimeout)
-					.map(new Mapper<ActorRef, ActorGateway>() {
-						@Override
-						public ActorGateway apply(ActorRef actorRef) {
-							return new AkkaActorGateway(actorRef, leaderSessionID);
-						}
-					}, actorSystem.dispatcher());
-		}
-	}
-
-	@Override
-	public void handleError(Exception exception) {
-		jobManagerFuture = Futures.failed(exception);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Retry strategy for failed lookups.
-	 *
-	 * <p>Usage:
-	 * <pre>
-	 * LookupRetryStrategy retryStrategy = LookupRetryStrategyFactory.create();
-	 *
-	 * if (retryStrategy.tryRetry()) {
-	 *     // OK to retry
-	 *     FiniteDuration retryDelay = retryStrategy.getRetryDelay();
-	 * }
-	 * </pre>
-	 */
-	interface LookupRetryStrategy {
-
-		/**
-		 * Returns the current retry.
-		 *
-		 * @return Current retry delay.
-		 */
-		FiniteDuration getRetryDelay();
-
-		/**
-		 * Tries another retry and returns whether it is allowed or not.
-		 *
-		 * @return Whether it is allowed to do another restart or not.
-		 */
-		boolean tryRetry();
-
-	}
-
-	/**
-	 * Factory for retry strategies.
-	 */
-	interface LookupRetryStrategyFactory {
-
-		/**
-		 * Creates a new retry strategy.
-		 *
-		 * @return The retry strategy.
-		 */
-		LookupRetryStrategy createRetryStrategy();
-
-	}
-
-	/**
-	 * Factory for disabled retries.
-	 */
-	static class DisabledLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
-
-		private static final DisabledLookupRetryStrategy RETRY_STRATEGY = new DisabledLookupRetryStrategy();
-
-		@Override
-		public LookupRetryStrategy createRetryStrategy() {
-			return RETRY_STRATEGY;
-		}
-
-		private static class DisabledLookupRetryStrategy implements LookupRetryStrategy {
-
-			@Override
-			public FiniteDuration getRetryDelay() {
-				return FiniteDuration.Zero();
-			}
-
-			@Override
-			public boolean tryRetry() {
-				return false;
-			}
-		}
-
-	}
-
-	/**
-	 * Factory for fixed delay retries.
-	 */
-	static class FixedDelayLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
-
-		private final int maxRetries;
-		private final FiniteDuration retryDelay;
-
-		FixedDelayLookupRetryStrategyFactory(int maxRetries, FiniteDuration retryDelay) {
-			this.maxRetries = maxRetries;
-			this.retryDelay = retryDelay;
-		}
-
-		@Override
-		public LookupRetryStrategy createRetryStrategy() {
-			return new FixedDelayLookupRetryStrategy(maxRetries, retryDelay);
-		}
-
-		private static class FixedDelayLookupRetryStrategy implements LookupRetryStrategy {
-
-			private final Object retryLock = new Object();
-			private final int maxRetries;
-			private final FiniteDuration retryDelay;
-			private int numRetries;
-
-			public FixedDelayLookupRetryStrategy(int maxRetries, FiniteDuration retryDelay) {
-				Preconditions.checkArgument(maxRetries >= 0, "Negative number maximum retries");
-				this.maxRetries = maxRetries;
-				this.retryDelay = Preconditions.checkNotNull(retryDelay, "Retry delay");
-			}
-
-			@Override
-			public FiniteDuration getRetryDelay() {
-				synchronized (retryLock) {
-					return retryDelay;
-				}
-			}
-
-			@Override
-			public boolean tryRetry() {
-				synchronized (retryLock) {
-					if (numRetries < maxRetries) {
-						numRetries++;
-						return true;
-					} else {
-						return false;
-					}
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 86d1838..8a213bb 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
@@ -31,7 +31,7 @@ import java.util.Arrays;
  * 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.
+ * Queryable State Client to target queries.
  */
 public class KvStateLocation implements Serializable {
 
@@ -166,7 +166,7 @@ public class KvStateLocation implements Serializable {
 	 * @param kvStateAddress Server address of the KvState instance at the key group index.
 	 * @throws IndexOutOfBoundsException If key group range start < 0 or key group range end >= Number of key groups
 	 */
-	void registerKvState(KeyGroupRange keyGroupRange, KvStateID kvStateId, KvStateServerAddress kvStateAddress) {
+	public void registerKvState(KeyGroupRange keyGroupRange, KvStateID kvStateId, KvStateServerAddress kvStateAddress) {
 
 		if (keyGroupRange.getStartKeyGroup() < 0 || keyGroupRange.getEndKeyGroup() >= numKeyGroups) {
 			throw new IndexOutOfBoundsException("Key group index");
@@ -183,6 +183,10 @@ public class KvStateLocation implements Serializable {
 		}
 	}
 
+	public static long getSerialVersionUID() {
+		return serialVersionUID;
+	}
+
 	/**
 	 * Registers a KvState instance for the given key group index.
 	 *
@@ -190,7 +194,7 @@ public class KvStateLocation implements Serializable {
 	 * @throws IndexOutOfBoundsException If key group range start < 0 or key group range end >= Number of key groups
 	 * @throws IllegalArgumentException  If no location information registered for a key group index in the range.
 	 */
-	void unregisterKvState(KeyGroupRange keyGroupRange) {
+	public void unregisterKvState(KeyGroupRange keyGroupRange) {
 		if (keyGroupRange.getStartKeyGroup() < 0 || keyGroupRange.getEndKeyGroup() >= numKeyGroups) {
 			throw new IndexOutOfBoundsException("Key group index");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationLookupService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationLookupService.java
deleted file mode 100644
index dfd9c14..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateLocationLookupService.java
+++ /dev/null
@@ -1,50 +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.query;
-
-import org.apache.flink.api.common.JobID;
-
-import scala.concurrent.Future;
-
-/**
- * {@link KvStateLocation} lookup service.
- */
-public interface KvStateLocationLookupService {
-
-	/**
-	 * Starts the lookup service.
-	 */
-	void start();
-
-	/**
-	 * Shuts down the lookup service.
-	 */
-	void shutDown();
-
-	/**
-	 * Returns a future holding the {@link KvStateLocation} for the given job
-	 * and KvState registration name.
-	 *
-	 * @param jobId            JobID the KvState instance belongs to
-	 * @param registrationName Name under which the KvState has been registered
-	 * @return Future holding the {@link KvStateLocation}
-	 */
-	Future<KvStateLocation> getKvStateLookupInfo(JobID jobId, String registrationName);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 26b700c..90fa5cc 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
@@ -20,7 +20,6 @@ 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.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.taskmanager.Task;

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
new file mode 100644
index 0000000..9b14c49
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.query;
+
+/**
+ * An interface for the Queryable State Server running on each Task Manager in the cluster.
+ * This server is responsible for serving requests coming from the Queryable State Client and
+ * requesting <b>locally</b> stored state.
+ */
+public interface KvStateServer {
+
+	/**
+	 * Returns the address of this server.
+	 *
+	 * @return Server address
+	 */
+	KvStateServerAddress getAddress();
+
+
+	/** Starts the proxy. */
+	void start() throws InterruptedException;
+
+	/**
+	 * Shuts down the server and all related thread pools.
+	 */
+	void shutDown();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java
index 9ec25bc..2599855 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateServerAddress.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.query;
 
-import org.apache.flink.runtime.query.netty.KvStateServer;
 import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
@@ -88,4 +87,9 @@ public class KvStateServerAddress implements Serializable {
 		result = 31 * result + port;
 		return result;
 	}
+
+	@Override
+	public String toString() {
+		return hostAddress.getHostName() + ':' + port;
+	}
 }


[07/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
new file mode 100644
index 0000000..eb33bce
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.queryablestate.messages;
+
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A {@link InternalKvState} instance request for a specific key and namespace.
+ */
+public final class KvStateRequest {
+
+	/** ID for this request. */
+	private final long requestId;
+
+	/** ID of the requested KvState instance. */
+	private final KvStateID kvStateId;
+
+	/** Serialized key and namespace to request from the KvState instance. */
+	private final byte[] serializedKeyAndNamespace;
+
+	/**
+	 * Creates a KvState instance request.
+	 *
+	 * @param requestId                 ID for this request
+	 * @param kvStateId                 ID of the requested KvState instance
+	 * @param serializedKeyAndNamespace Serialized key and namespace to request from the KvState
+	 *                                  instance
+	 */
+	public KvStateRequest(long requestId, KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
+		this.requestId = requestId;
+		this.kvStateId = Preconditions.checkNotNull(kvStateId, "KvStateID");
+		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
+	}
+
+	/**
+	 * Returns the request ID.
+	 *
+	 * @return Request ID
+	 */
+	public long getRequestId() {
+		return requestId;
+	}
+
+	/**
+	 * Returns the ID of the requested KvState instance.
+	 *
+	 * @return ID of the requested KvState instance
+	 */
+	public KvStateID getKvStateId() {
+		return kvStateId;
+	}
+
+	/**
+	 * Returns the serialized key and namespace to request from the KvState
+	 * instance.
+	 *
+	 * @return Serialized key and namespace to request from the KvState instance
+	 */
+	public byte[] getSerializedKeyAndNamespace() {
+		return serializedKeyAndNamespace;
+	}
+
+	@Override
+	public String toString() {
+		return "KvStateRequest{" +
+				"requestId=" + requestId +
+				", kvStateId=" + kvStateId +
+				", serializedKeyAndNamespace.length=" + serializedKeyAndNamespace.length +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
new file mode 100644
index 0000000..4015d79
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestFailure.java
@@ -0,0 +1,68 @@
+/*
+ * 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.queryablestate.messages;
+
+/**
+ * A failure response to a {@link KvStateRequest}.
+ */
+public final class KvStateRequestFailure {
+
+	/** ID of the request responding to. */
+	private final long requestId;
+
+	/** Failure cause. Not allowed to be a user type. */
+	private final Throwable cause;
+
+	/**
+	 * Creates a failure response to a {@link KvStateRequest}.
+	 *
+	 * @param requestId ID for the request responding to
+	 * @param cause     Failure cause (not allowed to be a user type)
+	 */
+	public KvStateRequestFailure(long requestId, Throwable cause) {
+		this.requestId = requestId;
+		this.cause = cause;
+	}
+
+	/**
+	 * Returns the request ID responding to.
+	 *
+	 * @return Request ID responding to
+	 */
+	public long getRequestId() {
+		return requestId;
+	}
+
+	/**
+	 * Returns the failure cause.
+	 *
+	 * @return Failure cause
+	 */
+	public Throwable getCause() {
+		return cause;
+	}
+
+	@Override
+	public String toString() {
+		return "KvStateRequestFailure{" +
+				"requestId=" + requestId +
+				", cause=" + cause +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
new file mode 100644
index 0000000..6bf2397
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/messages/KvStateRequestResult.java
@@ -0,0 +1,74 @@
+/*
+ * 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.queryablestate.messages;
+
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A successful response to a {@link KvStateRequest} containing the serialized
+ * result for the requested key and namespace.
+ */
+public final class KvStateRequestResult {
+
+	/** ID of the request responding to. */
+	private final long requestId;
+
+	/**
+	 * Serialized result for the requested key and namespace. If no result was
+	 * available for the specified key and namespace, this is <code>null</code>.
+	 */
+	private final byte[] serializedResult;
+
+	/**
+	 * Creates a successful {@link KvStateRequestResult} response.
+	 *
+	 * @param requestId        ID of the request responding to
+	 * @param serializedResult Serialized result or <code>null</code> if none
+	 */
+	public KvStateRequestResult(long requestId, byte[] serializedResult) {
+		this.requestId = requestId;
+		this.serializedResult = Preconditions.checkNotNull(serializedResult, "Serialization result");
+	}
+
+	/**
+	 * Returns the request ID responding to.
+	 *
+	 * @return Request ID responding to
+	 */
+	public long getRequestId() {
+		return requestId;
+	}
+
+	/**
+	 * Returns the serialized result or <code>null</code> if none available.
+	 *
+	 * @return Serialized result or <code>null</code> if none available.
+	 */
+	public byte[] getSerializedResult() {
+		return serializedResult;
+	}
+
+	@Override
+	public String toString() {
+		return "KvStateRequestResult{" +
+				"requestId=" + requestId +
+				", serializedResult.length=" + serializedResult.length +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
new file mode 100644
index 0000000..32bca64
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
@@ -0,0 +1,332 @@
+/*
+ * 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.queryablestate.network.messages;
+
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
+import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.util.AbstractID;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufInputStream;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufOutputStream;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+
+/**
+ * Serialization and deserialization of messages exchanged between
+ * {@link org.apache.flink.queryablestate.client.KvStateClient client} and
+ * {@link org.apache.flink.queryablestate.server.KvStateServerImpl server}.
+ *
+ * <p>The binary messages have the following format:
+ *
+ * <pre>
+ *                     <------ Frame ------------------------->
+ *                    +----------------------------------------+
+ *                    |        HEADER (8)      | PAYLOAD (VAR) |
+ * +------------------+----------------------------------------+
+ * | FRAME LENGTH (4) | VERSION (4) | TYPE (4) | CONTENT (VAR) |
+ * +------------------+----------------------------------------+
+ * </pre>
+ *
+ * <p>The concrete content of a message depends on the {@link MessageType}.
+ */
+public final class MessageSerializer {
+
+	/** The serialization version ID. */
+	private static final int VERSION = 0x79a1b710;
+
+	/** Byte length of the header. */
+	private static final int HEADER_LENGTH = 2 * Integer.BYTES;
+
+	/** Byte length of the request id. */
+	private static final int REQUEST_ID_SIZE = Long.BYTES;
+
+	// ------------------------------------------------------------------------
+	// Serialization
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Allocates a buffer and serializes the KvState request into it.
+	 *
+	 * @param alloc                     ByteBuf allocator for the buffer to
+	 *                                  serialize message into
+	 * @param requestId                 ID for this request
+	 * @param kvStateId                 ID of the requested KvState instance
+	 * @param serializedKeyAndNamespace Serialized key and namespace to request
+	 *                                  from the KvState instance.
+	 * @return Serialized KvState request message
+	 */
+	public static ByteBuf serializeKvStateRequest(
+			ByteBufAllocator alloc,
+			long requestId,
+			KvStateID kvStateId,
+			byte[] serializedKeyAndNamespace) {
+
+		// Header + request ID + KvState ID + Serialized namespace
+		int frameLength = HEADER_LENGTH + REQUEST_ID_SIZE + AbstractID.SIZE + (Integer.BYTES + serializedKeyAndNamespace.length);
+		ByteBuf buf = alloc.ioBuffer(frameLength + 4); // +4 for frame length
+
+		buf.writeInt(frameLength);
+
+		writeHeader(buf, MessageType.REQUEST);
+
+		buf.writeLong(requestId);
+		buf.writeLong(kvStateId.getLowerPart());
+		buf.writeLong(kvStateId.getUpperPart());
+		buf.writeInt(serializedKeyAndNamespace.length);
+		buf.writeBytes(serializedKeyAndNamespace);
+
+		return buf;
+	}
+
+	/**
+	 * Allocates a buffer and serializes the KvState request result into it.
+	 *
+	 * @param alloc             ByteBuf allocator for the buffer to serialize message into
+	 * @param requestId         ID for this request
+	 * @param serializedResult  Serialized Result
+	 * @return Serialized KvState request result message
+	 */
+	public static ByteBuf serializeKvStateRequestResult(
+			ByteBufAllocator alloc,
+			long requestId,
+			byte[] serializedResult) {
+
+		Preconditions.checkNotNull(serializedResult, "Serialized result");
+
+		// Header + request ID + serialized result
+		int frameLength = HEADER_LENGTH + REQUEST_ID_SIZE + 4 + serializedResult.length;
+
+		// TODO: 10/5/17 there was a bug all this time?
+		ByteBuf buf = alloc.ioBuffer(frameLength + 4);
+
+		buf.writeInt(frameLength);
+		writeHeader(buf, MessageType.REQUEST_RESULT);
+		buf.writeLong(requestId);
+
+		buf.writeInt(serializedResult.length);
+		buf.writeBytes(serializedResult);
+
+		return buf;
+	}
+
+	/**
+	 * Serializes the exception containing the failure message sent to the
+	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * protocol related errors.
+	 *
+	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
+	 * @param requestId		The id of the request to which the message refers to.
+	 * @param cause			The exception thrown at the server.
+	 * @return A {@link ByteBuf} containing the serialized message.
+	 */
+	public static ByteBuf serializeKvStateRequestFailure(
+			final ByteBufAllocator alloc,
+			final long requestId,
+			final Throwable cause) throws IOException {
+
+		final ByteBuf buf = alloc.ioBuffer();
+
+		// Frame length is set at the end
+		buf.writeInt(0);
+		writeHeader(buf, MessageType.REQUEST_FAILURE);
+		buf.writeLong(requestId);
+
+		try (ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
+				ObjectOutput out = new ObjectOutputStream(bbos)) {
+			out.writeObject(cause);
+		}
+
+		// Set frame length
+		int frameLength = buf.readableBytes() - Integer.BYTES;
+		buf.setInt(0, frameLength);
+		return buf;
+	}
+
+	/**
+	 * Serializes the failure message sent to the
+	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * server related errors.
+	 *
+	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
+	 * @param cause			The exception thrown at the server.
+	 * @return		The failure message.
+	 */
+	public static ByteBuf serializeServerFailure(
+			final ByteBufAllocator alloc,
+			final Throwable cause) throws IOException {
+
+		final ByteBuf buf = alloc.ioBuffer();
+
+		// Frame length is set at end
+		buf.writeInt(0);
+		writeHeader(buf, MessageType.SERVER_FAILURE);
+
+		try (ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
+				ObjectOutput out = new ObjectOutputStream(bbos)) {
+			out.writeObject(cause);
+		}
+
+		// Set frame length
+		int frameLength = buf.readableBytes() - Integer.BYTES;
+		buf.setInt(0, frameLength);
+		return buf;
+	}
+
+	/**
+	 * Helper for serializing the header.
+	 *
+	 * @param buf         The {@link ByteBuf} to serialize the header into.
+	 * @param messageType The {@link MessageType} of the message this header refers to.
+	 */
+	private static void writeHeader(final ByteBuf buf, final MessageType messageType) {
+		buf.writeInt(VERSION);
+		buf.writeInt(messageType.ordinal());
+	}
+
+	// ------------------------------------------------------------------------
+	// Deserialization
+	// ------------------------------------------------------------------------
+
+	/**
+	 * De-serializes the header and returns the {@link MessageType}.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the header position.</b>
+	 * </pre>
+	 * @param buf						The {@link ByteBuf} containing the serialized header.
+	 * @return							The message type.
+	 * @throws IllegalStateException	If unexpected message version or message type.
+	 */
+	public static MessageType deserializeHeader(final ByteBuf buf) {
+
+		// checking the version
+		int version = buf.readInt();
+		Preconditions.checkState(version == VERSION,
+				"Version Mismatch:  Found " + version + ", Expected: " + VERSION + '.');
+
+		// fetching the message type
+		int msgType = buf.readInt();
+		MessageType[] values = MessageType.values();
+		Preconditions.checkState(msgType >= 0 && msgType <= values.length,
+				"Illegal message type with index " + msgType + '.');
+		return values[msgType];
+	}
+
+	/**
+	 * Deserializes the KvState request message.
+	 *
+	 * <p><strong>Important</strong>: the returned buffer is sliced from the
+	 * incoming ByteBuf stream and retained. Therefore, it needs to be recycled
+	 * by the consumer.
+	 *
+	 * @param buf Buffer to deserialize (expected to be positioned after header)
+	 * @return Deserialized KvStateRequest
+	 */
+	public static KvStateRequest deserializeKvStateRequest(ByteBuf buf) {
+		long requestId = buf.readLong();
+		KvStateID kvStateId = new KvStateID(buf.readLong(), buf.readLong());
+
+		// Serialized key and namespace
+		int length = buf.readInt();
+
+		if (length < 0) {
+			throw new IllegalArgumentException("Negative length for serialized key and namespace. " +
+					"This indicates a serialization error.");
+		}
+
+		// Copy the buffer in order to be able to safely recycle the ByteBuf
+		byte[] serializedKeyAndNamespace = new byte[length];
+		if (length > 0) {
+			buf.readBytes(serializedKeyAndNamespace);
+		}
+
+		return new KvStateRequest(requestId, kvStateId, serializedKeyAndNamespace);
+	}
+
+	/**
+	 * Deserializes the KvState request result.
+	 *
+	 * @param buf Buffer to deserialize (expected to be positioned after header)
+	 * @return Deserialized KvStateRequestResult
+	 */
+	public static KvStateRequestResult deserializeKvStateRequestResult(ByteBuf buf) {
+		long requestId = buf.readLong();
+
+		// Serialized KvState
+		int length = buf.readInt();
+
+		if (length < 0) {
+			throw new IllegalArgumentException("Negative length for serialized result. " +
+					"This indicates a serialization error.");
+		}
+
+		byte[] serializedValue = new byte[length];
+
+		if (length > 0) {
+			buf.readBytes(serializedValue);
+		}
+
+		return new KvStateRequestResult(requestId, serializedValue);
+	}
+
+	/**
+	 * De-serializes the {@link KvStateRequestFailure} sent to the
+	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * protocol related errors.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the correct position.</b>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized failure message.
+	 * @return		The failure message.
+	 */
+	public static KvStateRequestFailure deserializeKvStateRequestFailure(final ByteBuf buf) throws IOException, ClassNotFoundException {
+		long requestId = buf.readLong();
+
+		Throwable cause;
+		try (ByteBufInputStream bis = new ByteBufInputStream(buf);
+				ObjectInputStream in = new ObjectInputStream(bis)) {
+			cause = (Throwable) in.readObject();
+		}
+		return new KvStateRequestFailure(requestId, cause);
+	}
+
+	/**
+	 * De-serializes the failure message sent to the
+	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * server related errors.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the correct position.</b>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized failure message.
+	 * @return		The failure message.
+	 */
+	public static Throwable deserializeServerFailure(final ByteBuf buf) throws IOException, ClassNotFoundException {
+		try (ByteBufInputStream bis = new ByteBufInputStream(buf);
+				ObjectInputStream in = new ObjectInputStream(bis)) {
+			return (Throwable) in.readObject();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
new file mode 100644
index 0000000..4e4435d
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.network.messages;
+
+/**
+ * Expected message types during the communication between
+ * {@link org.apache.flink.queryablestate.client.KvStateClient state client} and
+ * {@link org.apache.flink.queryablestate.server.KvStateServerImpl state server}.
+ */
+public enum MessageType {
+
+	/** The message is a request. */
+	REQUEST,
+
+	/** The message is a successful response. */
+	REQUEST_RESULT,
+
+	/** The message indicates a protocol-related failure. */
+	REQUEST_FAILURE,
+
+	/** The message indicates a server failure. */
+	SERVER_FAILURE
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
new file mode 100644
index 0000000..f10969e
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.server;
+
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedInput;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+/**
+ * A {@link ByteBuf} instance to be consumed in chunks by {@link ChunkedWriteHandler},
+ * respecting the high and low watermarks.
+ *
+ * @see <a href="http://normanmaurer.me/presentations/2014-facebook-eng-netty/slides.html#10.0">Low/High Watermarks</a>
+ */
+public class ChunkedByteBuf implements ChunkedInput<ByteBuf> {
+
+	/** The buffer to chunk. */
+	private final ByteBuf buf;
+
+	/** Size of chunks. */
+	private final int chunkSize;
+
+	/** Closed flag. */
+	private boolean isClosed;
+
+	/** End of input flag. */
+	private boolean isEndOfInput;
+
+	public ChunkedByteBuf(ByteBuf buf, int chunkSize) {
+		this.buf = Preconditions.checkNotNull(buf, "Buffer");
+		Preconditions.checkArgument(chunkSize > 0, "Non-positive chunk size");
+		this.chunkSize = chunkSize;
+	}
+
+	@Override
+	public boolean isEndOfInput() throws Exception {
+		return isClosed || isEndOfInput;
+	}
+
+	@Override
+	public void close() throws Exception {
+		if (!isClosed) {
+			// If we did not consume the whole buffer yet, we have to release
+			// it here. Otherwise, it's the responsibility of the consumer.
+			if (!isEndOfInput) {
+				buf.release();
+			}
+
+			isClosed = true;
+		}
+	}
+
+	@Override
+	public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception {
+		if (isClosed) {
+			return null;
+		} else if (buf.readableBytes() <= chunkSize) {
+			isEndOfInput = true;
+
+			// Don't retain as the consumer is responsible to release it
+			return buf.slice();
+		} else {
+			// Return a chunk sized slice of the buffer. The ref count is
+			// shared with the original buffer. That's why we need to retain
+			// a reference here.
+			return buf.readSlice(chunkSize).retain();
+		}
+	}
+
+	@Override
+	public String toString() {
+		return "ChunkedByteBuf{" +
+				"buf=" + buf +
+				", chunkSize=" + chunkSize +
+				", isClosed=" + isClosed +
+				", isEndOfInput=" + isEndOfInput +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
new file mode 100644
index 0000000..9a31fca
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
@@ -0,0 +1,308 @@
+/*
+ * 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.queryablestate.server;
+
+import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.queryablestate.UnknownKvStateID;
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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
+ * query task. The actual query is handled in a separate thread as it might
+ * otherwise block the network threads (file I/O etc.).
+ */
+@ChannelHandler.Sharable
+public class KvStateServerHandler extends ChannelInboundHandlerAdapter {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerHandler.class);
+
+	/** KvState registry holding references to the KvState instances. */
+	private final KvStateRegistry registry;
+
+	/** Thread pool for query execution. */
+	private final ExecutorService queryExecutor;
+
+	/** Exposed server statistics. */
+	private final KvStateRequestStats stats;
+
+	/**
+	 * Create the handler.
+	 *
+	 * @param kvStateRegistry Registry to query.
+	 * @param queryExecutor   Thread pool for query execution.
+	 * @param stats           Exposed server statistics.
+	 */
+	public KvStateServerHandler(
+			KvStateRegistry kvStateRegistry,
+			ExecutorService queryExecutor,
+			KvStateRequestStats stats) {
+
+		this.registry = Objects.requireNonNull(kvStateRegistry, "KvStateRegistry");
+		this.queryExecutor = Objects.requireNonNull(queryExecutor, "Query thread pool");
+		this.stats = Objects.requireNonNull(stats, "KvStateRequestStats");
+	}
+
+	@Override
+	public void channelActive(ChannelHandlerContext ctx) throws Exception {
+		stats.reportActiveConnection();
+	}
+
+	@Override
+	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+		stats.reportInactiveConnection();
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		KvStateRequest request = null;
+
+		try {
+			ByteBuf buf = (ByteBuf) msg;
+			MessageType msgType = MessageSerializer.deserializeHeader(buf);
+
+			if (msgType == MessageType.REQUEST) {
+				// ------------------------------------------------------------
+				// Request
+				// ------------------------------------------------------------
+				request = MessageSerializer.deserializeKvStateRequest(buf);
+
+				stats.reportRequest();
+
+				InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
+
+				if (kvState != null) {
+					// Execute actual query async, because it is possibly
+					// blocking (e.g. file I/O).
+					//
+					// A submission failure is not treated as fatal.
+					queryExecutor.submit(new AsyncKvStateQueryTask(ctx, request, kvState, stats));
+				} else {
+					ByteBuf unknown = MessageSerializer.serializeKvStateRequestFailure(
+							ctx.alloc(),
+							request.getRequestId(),
+							new UnknownKvStateID(request.getKvStateId()));
+
+					ctx.writeAndFlush(unknown);
+
+					stats.reportFailedRequest();
+				}
+			} else {
+				// ------------------------------------------------------------
+				// Unexpected
+				// ------------------------------------------------------------
+				ByteBuf failure = MessageSerializer.serializeServerFailure(
+						ctx.alloc(),
+						new IllegalArgumentException("Unexpected message type " + msgType
+								+ ". KvStateServerHandler expects "
+								+ MessageType.REQUEST + " messages."));
+
+				ctx.writeAndFlush(failure);
+			}
+		} catch (Throwable t) {
+			String stringifiedCause = ExceptionUtils.stringifyException(t);
+
+			ByteBuf err;
+			if (request != null) {
+				String errMsg = "Failed to handle incoming request with ID " +
+						request.getRequestId() + ". Caused by: " + stringifiedCause;
+				err = MessageSerializer.serializeKvStateRequestFailure(
+						ctx.alloc(),
+						request.getRequestId(),
+						new RuntimeException(errMsg));
+
+				stats.reportFailedRequest();
+			} else {
+				String errMsg = "Failed to handle incoming message. Caused by: " + stringifiedCause;
+				err = MessageSerializer.serializeServerFailure(
+						ctx.alloc(),
+						new RuntimeException(errMsg));
+			}
+
+			ctx.writeAndFlush(err);
+		} finally {
+			// IMPORTANT: We have to always recycle the incoming buffer.
+			// Otherwise we will leak memory out of Netty's buffer pool.
+			//
+			// If any operation ever holds on to the buffer, it is the
+			// responsibility of that operation to retain the buffer and
+			// release it later.
+			ReferenceCountUtil.release(msg);
+		}
+	}
+
+	@Override
+	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+		String stringifiedCause = ExceptionUtils.stringifyException(cause);
+		String msg = "Exception in server pipeline. Caused by: " + stringifiedCause;
+
+		ByteBuf err = MessageSerializer.serializeServerFailure(
+				ctx.alloc(),
+				new RuntimeException(msg));
+
+		ctx.writeAndFlush(err).addListener(ChannelFutureListener.CLOSE);
+	}
+
+	/**
+	 * Task to execute the actual query against the {@link InternalKvState} instance.
+	 */
+	private static class AsyncKvStateQueryTask implements Runnable {
+
+		private final ChannelHandlerContext ctx;
+
+		private final KvStateRequest request;
+
+		private final InternalKvState<?> kvState;
+
+		private final KvStateRequestStats stats;
+
+		private final long creationNanos;
+
+		public AsyncKvStateQueryTask(
+				ChannelHandlerContext ctx,
+				KvStateRequest request,
+				InternalKvState<?> kvState,
+				KvStateRequestStats stats) {
+
+			this.ctx = Objects.requireNonNull(ctx, "Channel handler context");
+			this.request = Objects.requireNonNull(request, "State query");
+			this.kvState = Objects.requireNonNull(kvState, "KvState");
+			this.stats = Objects.requireNonNull(stats, "State query stats");
+			this.creationNanos = System.nanoTime();
+		}
+
+		@Override
+		public void run() {
+			boolean success = false;
+
+			try {
+				if (!ctx.channel().isActive()) {
+					return;
+				}
+
+				// Query the KvState instance
+				byte[] serializedKeyAndNamespace = request.getSerializedKeyAndNamespace();
+				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
+
+				if (serializedResult != null) {
+					// We found some data, success!
+					ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
+							ctx.alloc(),
+							request.getRequestId(),
+							serializedResult);
+
+					int highWatermark = ctx.channel().config().getWriteBufferHighWaterMark();
+
+					ChannelFuture write;
+					if (buf.readableBytes() <= highWatermark) {
+						write = ctx.writeAndFlush(buf);
+					} else {
+						write = ctx.writeAndFlush(new ChunkedByteBuf(buf, highWatermark));
+					}
+
+					write.addListener(new QueryResultWriteListener());
+
+					success = true;
+				} else {
+					// No data for the key/namespace. This is considered to be
+					// a failure.
+					ByteBuf unknownKey = MessageSerializer.serializeKvStateRequestFailure(
+							ctx.alloc(),
+							request.getRequestId(),
+							new UnknownKeyOrNamespace());
+
+					ctx.writeAndFlush(unknownKey);
+				}
+			} catch (Throwable t) {
+				try {
+					String stringifiedCause = ExceptionUtils.stringifyException(t);
+					String errMsg = "Failed to query state backend for query " +
+							request.getRequestId() + ". Caused by: " + stringifiedCause;
+
+					ByteBuf err = MessageSerializer.serializeKvStateRequestFailure(
+							ctx.alloc(), request.getRequestId(), new RuntimeException(errMsg));
+
+					ctx.writeAndFlush(err);
+				} catch (IOException e) {
+					LOG.error("Failed to respond with the error after failed to query state backend", e);
+				}
+			} finally {
+				if (!success) {
+					stats.reportFailedRequest();
+				}
+			}
+		}
+
+		@Override
+		public String toString() {
+			return "AsyncKvStateQueryTask{" +
+					", request=" + request +
+					", creationNanos=" + creationNanos +
+					'}';
+		}
+
+		/**
+		 * Callback after query result has been written.
+		 *
+		 * <p>Gathers stats and logs errors.
+		 */
+		private class QueryResultWriteListener implements ChannelFutureListener {
+
+			@Override
+			public void operationComplete(ChannelFuture future) throws Exception {
+				long durationNanos = System.nanoTime() - creationNanos;
+				long durationMillis = TimeUnit.MILLISECONDS.convert(durationNanos, TimeUnit.NANOSECONDS);
+
+				if (future.isSuccess()) {
+					stats.reportSuccessfulRequest(durationMillis);
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Query " + request + " failed after " + durationMillis + " ms", future.cause());
+					}
+
+					stats.reportFailedRequest();
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
new file mode 100644
index 0000000..4bf7e24
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
@@ -0,0 +1,230 @@
+/*
+ * 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.queryablestate.server;
+
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Netty-based server answering {@link KvStateRequest} messages.
+ *
+ * <p>Requests are handled by asynchronous query tasks (see {@link KvStateServerHandler.AsyncKvStateQueryTask})
+ * that are executed by a separate query Thread pool. This pool is shared among
+ * all TCP connections.
+ *
+ * <p>The incoming pipeline looks as follows:
+ * <pre>
+ * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
+ * </pre>
+ *
+ * <p>Received binary messages are expected to contain a frame length field. Netty's
+ * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
+ * giving it to our {@link KvStateServerHandler}.
+ *
+ * <p>Connections are established and closed by the client. The server only
+ * closes the connection on a fatal failure that cannot be recovered. A
+ * server-side connection close is considered a failure by the client.
+ */
+public class KvStateServerImpl implements KvStateServer {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateServer.class);
+
+	/** Server config: low water mark. */
+	private static final int LOW_WATER_MARK = 8 * 1024;
+
+	/** Server config: high water mark. */
+	private static final int HIGH_WATER_MARK = 32 * 1024;
+
+	/** Netty's ServerBootstrap. */
+	private final ServerBootstrap bootstrap;
+
+	/** Query executor thread pool. */
+	private final ExecutorService queryExecutor;
+
+	/** Address of this server. */
+	private KvStateServerAddress serverAddress;
+
+	/**
+	 * Creates the {@link KvStateServer}.
+	 *
+	 * <p>The server needs to be started via {@link #start()} in order to bind
+	 * to the configured bind address.
+	 *
+	 * @param bindAddress         Address to bind to
+	 * @param bindPort            Port to bind to. Pick random port if 0.
+	 * @param numEventLoopThreads Number of event loop threads
+	 * @param numQueryThreads     Number of query threads
+	 * @param kvStateRegistry     KvStateRegistry to query for KvState instances
+	 * @param stats               Statistics tracker
+	 */
+	public KvStateServerImpl(
+			InetAddress bindAddress,
+			Integer bindPort,
+			Integer numEventLoopThreads,
+			Integer numQueryThreads,
+			KvStateRegistry kvStateRegistry,
+			KvStateRequestStats stats) {
+
+		Preconditions.checkArgument(bindPort >= 0 && bindPort <= 65536, "Port " + bindPort +
+				" is out of valid port range (0-65536).");
+
+		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
+		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
+
+		Preconditions.checkNotNull(kvStateRegistry, "KvStateRegistry");
+		Preconditions.checkNotNull(stats, "KvStateRequestStats");
+
+		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
+
+		ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink KvStateServer EventLoop Thread %d")
+				.build();
+
+		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
+
+		queryExecutor = createQueryExecutor(numQueryThreads);
+
+		// Shared between all channels
+		KvStateServerHandler serverHandler = new KvStateServerHandler(
+				kvStateRegistry,
+				queryExecutor,
+				stats);
+
+		bootstrap = new ServerBootstrap()
+				// Bind address and port
+				.localAddress(bindAddress, bindPort)
+				// NIO server channels
+				.group(nioGroup)
+				.channel(NioServerSocketChannel.class)
+				// Server channel Options
+				.option(ChannelOption.ALLOCATOR, bufferPool)
+				// Child channel options
+				.childOption(ChannelOption.ALLOCATOR, bufferPool)
+				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
+				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK)
+				// See initializer for pipeline details
+				.childHandler(new KvStateServerChannelInitializer(serverHandler));
+	}
+
+	@Override
+	public void start() throws InterruptedException {
+		Channel channel = bootstrap.bind().sync().channel();
+
+		InetSocketAddress localAddress = (InetSocketAddress) channel.localAddress();
+		serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
+	}
+
+	@Override
+	public KvStateServerAddress getAddress() {
+		if (serverAddress == null) {
+			throw new IllegalStateException("KvStateServer not started yet.");
+		}
+
+		return serverAddress;
+	}
+
+	@Override
+	public void shutDown() {
+		if (bootstrap != null) {
+			EventLoopGroup group = bootstrap.group();
+			if (group != null) {
+				group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
+			}
+		}
+
+		if (queryExecutor != null) {
+			queryExecutor.shutdown();
+		}
+
+		serverAddress = null;
+	}
+
+	/**
+	 * Creates a thread pool for the query execution.
+	 *
+	 * @param numQueryThreads Number of query threads.
+	 * @return Thread pool for query execution
+	 */
+	private static ExecutorService createQueryExecutor(int numQueryThreads) {
+		ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink KvStateServer Query Thread %d")
+				.build();
+
+		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
+	}
+
+	/**
+	 * Channel pipeline initializer.
+	 *
+	 * <p>The request handler is shared, whereas the other handlers are created
+	 * per channel.
+	 */
+	private static final class KvStateServerChannelInitializer extends ChannelInitializer<SocketChannel> {
+
+		/** The shared request handler. */
+		private final KvStateServerHandler sharedRequestHandler;
+
+		/**
+		 * Creates the channel pipeline initializer with the shared request handler.
+		 *
+		 * @param sharedRequestHandler Shared request handler.
+		 */
+		public KvStateServerChannelInitializer(KvStateServerHandler sharedRequestHandler) {
+			this.sharedRequestHandler = Preconditions.checkNotNull(sharedRequestHandler, "Request handler");
+		}
+
+		@Override
+		protected void initChannel(SocketChannel ch) throws Exception {
+			ch.pipeline()
+					.addLast(new ChunkedWriteHandler())
+					.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+					.addLast(sharedRequestHandler);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
new file mode 100644
index 0000000..a7f65f3
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
@@ -0,0 +1,1128 @@
+/*
+ * 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.queryablestate.itcases;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.queryablestate.client.QueryableStateClient;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
+import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceTypeInfo;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.QueryableStateStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
+import akka.dispatch.OnSuccess;
+import akka.dispatch.Recover;
+import akka.pattern.Patterns;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongArray;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+import scala.reflect.ClassTag$;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Base class for queryable state integration tests with a configurable state backend.
+ */
+public abstract class AbstractQueryableStateITCase extends TestLogger {
+
+	protected static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000, TimeUnit.SECONDS);
+	private static final FiniteDuration QUERY_RETRY_DELAY = new FiniteDuration(100, TimeUnit.MILLISECONDS);
+
+	protected static ActorSystem testActorSystem;
+
+	/**
+	 * State backend to use.
+	 */
+	protected AbstractStateBackend stateBackend;
+
+	/**
+	 * Shared between all the test. Make sure to have at least NUM_SLOTS
+	 * available after your test finishes, e.g. cancel the job you submitted.
+	 */
+	protected static FlinkMiniCluster cluster;
+
+	protected static int maxParallelism;
+
+	@Before
+	public void setUp() throws Exception {
+		// NOTE: do not use a shared instance for all tests as the tests may brake
+		this.stateBackend = createStateBackend();
+
+		Assert.assertNotNull(cluster);
+
+		maxParallelism = cluster.configuration().getInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1) *
+				cluster.configuration().getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+	}
+
+	/**
+	 * Creates a state backend instance which is used in the {@link #setUp()} method before each
+	 * test case.
+	 *
+	 * @return a state backend instance for each unit test
+	 */
+	protected abstract AbstractStateBackend createStateBackend() throws Exception;
+
+	/**
+	 * Runs a simple topology producing random (key, 1) pairs at the sources (where
+	 * number of keys is in fixed in range 0...numKeys). The records are keyed and
+	 * a reducing queryable state instance is created, which sums up the records.
+	 *
+	 * <p>After submitting the job in detached mode, the QueryableStateCLient is used
+	 * to query the counts of each key in rounds until all keys have non-zero counts.
+	 */
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testQueryableState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+		final int numKeys = 256;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+
+		try {
+			//
+			// Test program
+			//
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestKeyRangeSource(numKeys));
+
+			// Reducing state
+			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState = new ReducingStateDescriptor<>(
+					"any-name",
+					new SumReduce(),
+					source.getType());
+
+			final String queryName = "hakuna-matata";
+
+			final QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = 7143749578983540352L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState(queryName, reducingState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			cluster.submitJobDetached(jobGraph);
+
+			//
+			// Start querying
+			//
+			jobId = jobGraph.getJobID();
+
+			final AtomicLongArray counts = new AtomicLongArray(numKeys);
+
+			boolean allNonZero = false;
+			while (!allNonZero && deadline.hasTimeLeft()) {
+				allNonZero = true;
+
+				final List<Future<Tuple2<Integer, Long>>> futures = new ArrayList<>(numKeys);
+
+				for (int i = 0; i < numKeys; i++) {
+					final int key = i;
+
+					if (counts.get(key) > 0) {
+						// Skip this one
+						continue;
+					} else {
+						allNonZero = false;
+					}
+
+					Future<Tuple2<Integer, Long>> result = getKvStateWithRetries(
+							client,
+							jobId,
+							queryName,
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							reducingState,
+							QUERY_RETRY_DELAY,
+							false);
+
+					result.onSuccess(new OnSuccess<Tuple2<Integer, Long>>() {
+						@Override
+						public void onSuccess(Tuple2<Integer, Long> result) throws Throwable {
+							counts.set(key, result.f1);
+							assertEquals("Key mismatch", key, result.f0.intValue());
+						}
+					}, testActorSystem.dispatcher());
+
+					futures.add(result);
+				}
+
+				Future<Iterable<Tuple2<Integer, Long>>> futureSequence = Futures.sequence(
+						futures,
+						testActorSystem.dispatcher());
+
+				Await.ready(futureSequence, deadline.timeLeft());
+			}
+
+			assertTrue("Not all keys are non-zero", allNonZero);
+
+			// All should be non-zero
+			for (int i = 0; i < numKeys; i++) {
+				long count = counts.get(i);
+				assertTrue("Count at position " + i + " is " + count, count > 0);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Tests that duplicate query registrations fail the job at the JobManager.
+	 *
+	 * <b>NOTE: </b> This test is only in the non-HA variant of the tests because
+	 * in the HA mode we use the actual JM code which does not recognize the
+	 * {@code NotifyWhenJobStatus} message.	 *
+	 */
+	@Test
+	public void testDuplicateRegistrationFailsJob() throws Exception {
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+		final int numKeys = 256;
+
+		JobID jobId = null;
+
+		try {
+			//
+			// Test program
+			//
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestKeyRangeSource(numKeys));
+
+			// Reducing state
+			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState = new ReducingStateDescriptor<>(
+					"any-name",
+					new SumReduce(),
+					source.getType());
+
+			final String queryName = "duplicate-me";
+
+			final QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = -4126824763829132959L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState(queryName, reducingState);
+
+			final QueryableStateStream<Integer, Tuple2<Integer, Long>> duplicate =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = -6265024000462809436L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState(queryName);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			Future<TestingJobManagerMessages.JobStatusIs> failedFuture = cluster
+					.getLeaderGateway(deadline.timeLeft())
+					.ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.FAILED), deadline.timeLeft())
+					.mapTo(ClassTag$.MODULE$.<TestingJobManagerMessages.JobStatusIs>apply(TestingJobManagerMessages.JobStatusIs.class));
+
+			cluster.submitJobDetached(jobGraph);
+
+			TestingJobManagerMessages.JobStatusIs jobStatus = Await.result(failedFuture, deadline.timeLeft());
+			assertEquals(JobStatus.FAILED, jobStatus.state());
+
+			// Get the job and check the cause
+			JobManagerMessages.JobFound jobFound = Await.result(
+					cluster.getLeaderGateway(deadline.timeLeft())
+							.ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft())
+							.mapTo(ClassTag$.MODULE$.<JobManagerMessages.JobFound>apply(JobManagerMessages.JobFound.class)),
+					deadline.timeLeft());
+
+			String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString();
+
+			assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException"));
+			int causedByIndex = failureCause.indexOf("Caused by: ");
+			String subFailureCause = failureCause.substring(causedByIndex + "Caused by: ".length());
+			assertTrue("Not caused by IllegalStateException", subFailureCause.startsWith("java.lang.IllegalStateException"));
+			assertTrue("Exception does not contain registration name", subFailureCause.contains(queryName));
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+		}
+	}
+
+	/**
+	 * Tests simple value state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The tests succeeds after each subtask index is queried with
+	 * value numElements (the latest element updated the state).
+	 */
+	@Test
+	public void testValueState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Value state
+			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
+					"any",
+					source.getType());
+
+			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = 7662520075515707428L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState("hakuna", valueState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+			long expected = numElements;
+
+			executeQuery(deadline, client, jobId, "hakuna", valueState, expected);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Similar tests as {@link #testValueState()} but before submitting the
+	 * job, we already issue one request which fails.
+	 */
+	@Test
+	public void testQueryNonStartedJobState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+				.addSource(new TestAscendingValueSource(numElements));
+
+			// Value state
+			ValueStateDescriptor<Tuple2<Integer, Long>> valueState = new ValueStateDescriptor<>(
+				"any",
+				source.getType(),
+				null);
+
+			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+				source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+					private static final long serialVersionUID = 7480503339992214681L;
+
+					@Override
+					public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+						return value.f0;
+					}
+				}).asQueryableState("hakuna", valueState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			// Now query
+			long expected = numElements;
+
+			// query once
+			client.getKvState(
+					jobId,
+					queryableState.getQueryableStateName(),
+					0,
+					VoidNamespace.INSTANCE,
+					BasicTypeInfo.INT_TYPE_INFO,
+					VoidNamespaceTypeInfo.INSTANCE,
+					valueState);
+
+			cluster.submitJobDetached(jobGraph);
+
+			executeQuery(deadline, client, jobId, "hakuna", valueState, expected);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+					.getLeaderGateway(deadline.timeLeft())
+					.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Retry a query for state for keys between 0 and {@link #maxParallelism} until
+	 * <tt>expected</tt> equals the value of the result tuple's second field.
+	 */
+	private void executeQuery(
+			final Deadline deadline,
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryableStateName,
+			final StateDescriptor<?, Tuple2<Integer, Long>> stateDescriptor,
+			final long expected) throws Exception {
+
+		for (int key = 0; key < maxParallelism; key++) {
+			boolean success = false;
+			while (deadline.hasTimeLeft() && !success) {
+				Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
+					jobId,
+					queryableStateName,
+					key,
+					BasicTypeInfo.INT_TYPE_INFO,
+					stateDescriptor,
+					QUERY_RETRY_DELAY,
+					false);
+
+				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
+
+				assertEquals("Key mismatch", key, value.f0.intValue());
+				if (expected == value.f1) {
+					success = true;
+				} else {
+					// Retry
+					Thread.sleep(50);
+				}
+			}
+
+			assertTrue("Did not succeed query", success);
+		}
+	}
+
+	/**
+	 * Retry a query for state for keys between 0 and {@link #maxParallelism} until
+	 * <tt>expected</tt> equals the value of the result tuple's second field.
+	 */
+	private void executeQuery(
+			final Deadline deadline,
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryableStateName,
+			final TypeSerializer<Tuple2<Integer, Long>> valueSerializer,
+			final long expected) throws Exception {
+
+		for (int key = 0; key < maxParallelism; key++) {
+			boolean success = false;
+			while (deadline.hasTimeLeft() && !success) {
+				Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
+						jobId,
+						queryableStateName,
+						key,
+						BasicTypeInfo.INT_TYPE_INFO,
+						valueSerializer,
+						QUERY_RETRY_DELAY,
+						false);
+
+				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
+
+				assertEquals("Key mismatch", key, value.f0.intValue());
+				if (expected == value.f1) {
+					success = true;
+				} else {
+					// Retry
+					Thread.sleep(50);
+				}
+			}
+
+			assertTrue("Did not succeed query", success);
+		}
+	}
+
+	/**
+	 * Tests simple value state queryable state instance with a default value
+	 * set. Each source emits (subtaskIndex, 0)..(subtaskIndex, numElements)
+	 * tuples, the key is mapped to 1 but key 0 is queried which should throw
+	 * a {@link UnknownKeyOrNamespace} exception.
+	 *
+	 * @throws UnknownKeyOrNamespace thrown due querying a non-existent key
+	 */
+	@Test(expected = UnknownKeyOrNamespace.class)
+	public void testValueStateDefault() throws
+		Exception, UnknownKeyOrNamespace {
+
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies
+				.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+				.addSource(new TestAscendingValueSource(numElements));
+
+			// Value state
+			ValueStateDescriptor<Tuple2<Integer, Long>> valueState =
+				new ValueStateDescriptor<>(
+					"any",
+					source.getType(),
+					Tuple2.of(0, 1337L));
+
+			// only expose key "1"
+			QueryableStateStream<Integer, Tuple2<Integer, Long>>
+				queryableState =
+				source.keyBy(
+					new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = 4509274556892655887L;
+
+						@Override
+						public Integer getKey(
+							Tuple2<Integer, Long> value) throws
+							Exception {
+							return 1;
+						}
+					}).asQueryableState("hakuna", valueState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+			int key = 0;
+			Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
+				jobId,
+				queryableState.getQueryableStateName(),
+				key,
+				BasicTypeInfo.INT_TYPE_INFO,
+				valueState,
+				QUERY_RETRY_DELAY,
+				true);
+
+			Await.result(future, deadline.timeLeft());
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+					.getLeaderGateway(deadline.timeLeft())
+					.ask(new JobManagerMessages.CancelJob(jobId),
+						deadline.timeLeft())
+					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(
+						CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Tests simple value state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The tests succeeds after each subtask index is queried with
+	 * value numElements (the latest element updated the state).
+	 *
+	 * <p>This is the same as the simple value state test, but uses the API shortcut.
+	 */
+	@Test
+	public void testValueStateShortcut() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Value state shortcut
+			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = 9168901838808830068L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState("matata");
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+			long expected = numElements;
+
+			executeQuery(deadline, client, jobId, "matata",
+					queryableState.getValueSerializer(), expected);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Tests simple folding state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The folding state sums these up and maps them to Strings. The
+	 * test succeeds after each subtask index is queried with result n*(n+1)/2
+	 * (as a String).
+	 */
+	@Test
+	public void testFoldingState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Folding state
+			FoldingStateDescriptor<Tuple2<Integer, Long>, String> foldingState =
+					new FoldingStateDescriptor<>(
+							"any",
+							"0",
+							new SumFold(),
+							StringSerializer.INSTANCE);
+
+			QueryableStateStream<Integer, String> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = -842809958106747539L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState("pumba", foldingState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Now query
+			String expected = Integer.toString(numElements * (numElements + 1) / 2);
+
+			for (int key = 0; key < maxParallelism; key++) {
+				boolean success = false;
+				while (deadline.hasTimeLeft() && !success) {
+					Future<String> future = getKvStateWithRetries(client,
+							jobId,
+							queryableState.getQueryableStateName(),
+							key,
+							BasicTypeInfo.INT_TYPE_INFO,
+							foldingState,
+							QUERY_RETRY_DELAY,
+							false);
+
+					String value = Await.result(future, deadline.timeLeft());
+					if (expected.equals(value)) {
+						success = true;
+					} else {
+						// Retry
+						Thread.sleep(50);
+					}
+				}
+
+				assertTrue("Did not succeed query", success);
+			}
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	/**
+	 * Tests simple reducing state queryable state instance. Each source emits
+	 * (subtaskIndex, 0)..(subtaskIndex, numElements) tuples, which are then
+	 * queried. The reducing state instance sums these up. The test succeeds
+	 * after each subtask index is queried with result n*(n+1)/2.
+	 */
+	@Test
+	public void testReducingState() throws Exception {
+		// Config
+		final Deadline deadline = TEST_TIMEOUT.fromNow();
+
+		final int numElements = 1024;
+
+		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+
+		JobID jobId = null;
+		try {
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+			env.setStateBackend(stateBackend);
+			env.setParallelism(maxParallelism);
+			// Very important, because cluster is shared between tests and we
+			// don't explicitly check that all slots are available before
+			// submitting.
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+
+			DataStream<Tuple2<Integer, Long>> source = env
+					.addSource(new TestAscendingValueSource(numElements));
+
+			// Reducing state
+			ReducingStateDescriptor<Tuple2<Integer, Long>> reducingState =
+					new ReducingStateDescriptor<>(
+							"any",
+							new SumReduce(),
+							source.getType());
+
+			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
+					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+						private static final long serialVersionUID = 8470749712274833552L;
+
+						@Override
+						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+							return value.f0;
+						}
+					}).asQueryableState("jungle", reducingState);
+
+			// Submit the job graph
+			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
+			jobId = jobGraph.getJobID();
+
+			cluster.submitJobDetached(jobGraph);
+
+			// Wait until job is running
+
+			// Now query
+			long expected = numElements * (numElements + 1) / 2;
+
+			executeQuery(deadline, client, jobId, "jungle", reducingState, expected);
+		} finally {
+			// Free cluster resources
+			if (jobId != null) {
+				Future<CancellationSuccess> cancellation = cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+
+				Await.ready(cancellation, deadline.timeLeft());
+			}
+
+			client.shutDown();
+		}
+	}
+
+	private static <K, V> Future<V> getKvStateWithRetries(
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final TypeSerializer<V> valueTypeSerializer,
+			final FiniteDuration retryDelay,
+			final boolean failForUnknownKeyOrNamespace) {
+
+		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, valueTypeSerializer)
+				.recoverWith(new Recover<Future<V>>() {
+					@Override
+					public Future<V> recover(Throwable failure) throws Throwable {
+						if (failure instanceof AssertionError) {
+							return Futures.failed(failure);
+						} else if (failForUnknownKeyOrNamespace &&
+								(failure instanceof UnknownKeyOrNamespace)) {
+							return Futures.failed(failure);
+						} else {
+							// At startup some failures are expected
+							// due to races. Make sure that they don't
+							// fail this test.
+							return Patterns.after(
+									retryDelay,
+									testActorSystem.scheduler(),
+									testActorSystem.dispatcher(),
+									new Callable<Future<V>>() {
+										@Override
+										public Future<V> call() throws Exception {
+											return getKvStateWithRetries(
+													client,
+													jobId,
+													queryName,
+													key,
+													keyTypeInfo,
+													valueTypeSerializer,
+													retryDelay,
+													failForUnknownKeyOrNamespace);
+										}
+									});
+						}
+					}
+				}, testActorSystem.dispatcher());
+
+	}
+
+	private static <K, V> Future<V> getKvStateWithRetries(
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final StateDescriptor<?, V> stateDescriptor,
+			final FiniteDuration retryDelay,
+			final boolean failForUnknownKeyOrNamespace) {
+
+		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor)
+				.recoverWith(new Recover<Future<V>>() {
+					@Override
+					public Future<V> recover(Throwable failure) throws Throwable {
+						if (failure instanceof AssertionError) {
+							return Futures.failed(failure);
+						} else if (failForUnknownKeyOrNamespace &&
+								(failure instanceof UnknownKeyOrNamespace)) {
+							return Futures.failed(failure);
+						} else {
+							// At startup some failures are expected
+							// due to races. Make sure that they don't
+							// fail this test.
+							return Patterns.after(
+									retryDelay,
+									testActorSystem.scheduler(),
+									testActorSystem.dispatcher(),
+									new Callable<Future<V>>() {
+										@Override
+										public Future<V> call() throws Exception {
+											return getKvStateWithRetries(
+													client,
+													jobId,
+													queryName,
+													key,
+													keyTypeInfo,
+													stateDescriptor,
+													retryDelay,
+													failForUnknownKeyOrNamespace);
+										}
+									});
+						}
+					}
+				}, testActorSystem.dispatcher());
+	}
+
+	/**
+	 * Test source producing (key, 0)..(key, maxValue) with key being the sub
+	 * task index.
+	 *
+	 * <p>After all tuples have been emitted, the source waits to be cancelled
+	 * and does not immediately finish.
+	 */
+	private static class TestAscendingValueSource extends RichParallelSourceFunction<Tuple2<Integer, Long>> {
+
+		private static final long serialVersionUID = 1459935229498173245L;
+
+		private final long maxValue;
+		private volatile boolean isRunning = true;
+
+		TestAscendingValueSource(long maxValue) {
+			Preconditions.checkArgument(maxValue >= 0);
+			this.maxValue = maxValue;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, Long>> ctx) throws Exception {
+			// f0 => key
+			int key = getRuntimeContext().getIndexOfThisSubtask();
+			Tuple2<Integer, Long> record = new Tuple2<>(key, 0L);
+
+			long currentValue = 0;
+			while (isRunning && currentValue <= maxValue) {
+				synchronized (ctx.getCheckpointLock()) {
+					record.f1 = currentValue;
+					ctx.collect(record);
+				}
+
+				currentValue++;
+			}
+
+			while (isRunning) {
+				synchronized (this) {
+					this.wait();
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+
+			synchronized (this) {
+				this.notifyAll();
+			}
+		}
+
+	}
+
+	/**
+	 * Test source producing (key, 1) tuples with random key in key range (numKeys).
+	 */
+	protected static class TestKeyRangeSource extends RichParallelSourceFunction<Tuple2<Integer, Long>>
+			implements CheckpointListener {
+		private static final long serialVersionUID = -5744725196953582710L;
+
+		private static final AtomicLong LATEST_CHECKPOINT_ID = new AtomicLong();
+		private final int numKeys;
+		private final ThreadLocalRandom random = ThreadLocalRandom.current();
+		private volatile boolean isRunning = true;
+
+		TestKeyRangeSource(int numKeys) {
+			this.numKeys = numKeys;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+				LATEST_CHECKPOINT_ID.set(0);
+			}
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, Long>> ctx) throws Exception {
+			// f0 => key
+			Tuple2<Integer, Long> record = new Tuple2<>(0, 1L);
+
+			while (isRunning) {
+				synchronized (ctx.getCheckpointLock()) {
+					record.f0 = random.nextInt(numKeys);
+					ctx.collect(record);
+				}
+				// mild slow down
+				Thread.sleep(1);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) throws Exception {
+			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+				LATEST_CHECKPOINT_ID.set(checkpointId);
+			}
+		}
+	}
+
+	/**
+	 * Test {@link FoldFunction} concatenating the already stored string with the long passed as argument.
+	 */
+	private static class SumFold implements FoldFunction<Tuple2<Integer, Long>, String> {
+		private static final long serialVersionUID = -6249227626701264599L;
+
+		@Override
+		public String fold(String accumulator, Tuple2<Integer, Long> value) throws Exception {
+			long acc = Long.valueOf(accumulator);
+			acc += value.f1;
+			return Long.toString(acc);
+		}
+	}
+
+	/**
+	 * Test {@link ReduceFunction} summing up its two arguments.
+	 */
+	protected static class SumReduce implements ReduceFunction<Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = -8651235077342052336L;
+
+		@Override
+		public Tuple2<Integer, Long> reduce(Tuple2<Integer, Long> value1, Tuple2<Integer, Long> value2) throws Exception {
+			value1.f1 += value2.f1;
+			return value1;
+		}
+	}
+
+}


[11/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
index a7f65f3..7ff4ec6 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.common.state.FoldingStateDescriptor;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -34,8 +35,12 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.configuration.QueryableStateOptions;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespaceException;
 import org.apache.flink.queryablestate.client.QueryableStateClient;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.concurrent.ScheduledExecutor;
+import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.messages.JobManagerMessages;
@@ -53,25 +58,27 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunctio
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
 
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.OnSuccess;
-import akka.dispatch.Recover;
-import akka.pattern.Patterns;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLongArray;
+import java.util.function.Supplier;
 
 import scala.concurrent.Await;
-import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 import scala.reflect.ClassTag$;
@@ -84,10 +91,12 @@ import static org.junit.Assert.assertTrue;
  */
 public abstract class AbstractQueryableStateITCase extends TestLogger {
 
-	protected static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000, TimeUnit.SECONDS);
-	private static final FiniteDuration QUERY_RETRY_DELAY = new FiniteDuration(100, TimeUnit.MILLISECONDS);
+	private static final int NO_OF_RETRIES = 100;
+	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000L, TimeUnit.SECONDS);
+	private static final Time QUERY_RETRY_DELAY = Time.milliseconds(100L);
 
-	protected static ActorSystem testActorSystem;
+	private final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4);
+	private final ScheduledExecutor executor = new ScheduledExecutorServiceAdapter(executorService);
 
 	/**
 	 * State backend to use.
@@ -136,7 +145,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 		final int numKeys = 256;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 
@@ -150,7 +161,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Very important, because cluster is shared between tests and we
 			// don't explicitly check that all slots are available before
 			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 					.addSource(new TestKeyRangeSource(numKeys));
@@ -163,15 +174,14 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 			final String queryName = "hakuna-matata";
 
-			final QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 7143749578983540352L;
+			source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+				private static final long serialVersionUID = 7143749578983540352L;
 
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState(queryName, reducingState);
+				@Override
+				public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+					return value.f0;
+				}
+			}).asQueryableState(queryName, reducingState);
 
 			// Submit the job graph
 			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
@@ -188,19 +198,19 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			while (!allNonZero && deadline.hasTimeLeft()) {
 				allNonZero = true;
 
-				final List<Future<Tuple2<Integer, Long>>> futures = new ArrayList<>(numKeys);
+				final List<CompletableFuture<Tuple2<Integer, Long>>> futures = new ArrayList<>(numKeys);
 
 				for (int i = 0; i < numKeys; i++) {
 					final int key = i;
 
-					if (counts.get(key) > 0) {
+					if (counts.get(key) > 0L) {
 						// Skip this one
 						continue;
 					} else {
 						allNonZero = false;
 					}
 
-					Future<Tuple2<Integer, Long>> result = getKvStateWithRetries(
+					CompletableFuture<Tuple2<Integer, Long>> result = getKvStateWithRetries(
 							client,
 							jobId,
 							queryName,
@@ -208,24 +218,21 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							BasicTypeInfo.INT_TYPE_INFO,
 							reducingState,
 							QUERY_RETRY_DELAY,
-							false);
+							false,
+							executor);
 
-					result.onSuccess(new OnSuccess<Tuple2<Integer, Long>>() {
-						@Override
-						public void onSuccess(Tuple2<Integer, Long> result) throws Throwable {
-							counts.set(key, result.f1);
-							assertEquals("Key mismatch", key, result.f0.intValue());
-						}
-					}, testActorSystem.dispatcher());
+					result.thenAccept(res -> {
+						counts.set(key, res.f1);
+						assertEquals("Key mismatch", key, res.f0.intValue());
+					});
 
 					futures.add(result);
 				}
 
-				Future<Iterable<Tuple2<Integer, Long>>> futureSequence = Futures.sequence(
-						futures,
-						testActorSystem.dispatcher());
-
-				Await.ready(futureSequence, deadline.timeLeft());
+				// wait for all the futures to complete
+				CompletableFuture
+						.allOf(futures.toArray(new CompletableFuture<?>[futures.size()]))
+						.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
 			assertTrue("Not all keys are non-zero", allNonZero);
@@ -238,15 +245,15 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
 						.getLeaderGateway(deadline.timeLeft())
 						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -274,7 +281,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Very important, because cluster is shared between tests and we
 			// don't explicitly check that all slots are available before
 			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 					.addSource(new TestKeyRangeSource(numKeys));
@@ -311,22 +318,23 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
 			jobId = jobGraph.getJobID();
 
-			Future<TestingJobManagerMessages.JobStatusIs> failedFuture = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.FAILED), deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<TestingJobManagerMessages.JobStatusIs>apply(TestingJobManagerMessages.JobStatusIs.class));
+			CompletableFuture<TestingJobManagerMessages.JobStatusIs> failedFuture = FutureUtils.toJava(
+					cluster.getLeaderGateway(deadline.timeLeft())
+							.ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, JobStatus.FAILED), deadline.timeLeft())
+							.mapTo(ClassTag$.MODULE$.<TestingJobManagerMessages.JobStatusIs>apply(TestingJobManagerMessages.JobStatusIs.class)));
 
 			cluster.submitJobDetached(jobGraph);
 
-			TestingJobManagerMessages.JobStatusIs jobStatus = Await.result(failedFuture, deadline.timeLeft());
+			TestingJobManagerMessages.JobStatusIs jobStatus =
+					failedFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			assertEquals(JobStatus.FAILED, jobStatus.state());
 
 			// Get the job and check the cause
-			JobManagerMessages.JobFound jobFound = Await.result(
+			JobManagerMessages.JobFound jobFound = FutureUtils.toJava(
 					cluster.getLeaderGateway(deadline.timeLeft())
 							.ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft())
-							.mapTo(ClassTag$.MODULE$.<JobManagerMessages.JobFound>apply(JobManagerMessages.JobFound.class)),
-					deadline.timeLeft());
+							.mapTo(ClassTag$.MODULE$.<JobManagerMessages.JobFound>apply(JobManagerMessages.JobFound.class)))
+					.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 			String failureCause = jobFound.executionGraph().getFailureCause().getExceptionAsString();
 
@@ -338,10 +346,10 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
+				scala.concurrent.Future<CancellationSuccess> cancellation = cluster
 						.getLeaderGateway(deadline.timeLeft())
 						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+						.mapTo(ClassTag$.MODULE$.<JobManagerMessages.CancellationSuccess>apply(JobManagerMessages.CancellationSuccess.class));
 
 				Await.ready(cancellation, deadline.timeLeft());
 			}
@@ -359,9 +367,11 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		// Config
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 
-		final int numElements = 1024;
+		final long numElements = 1024L;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -371,7 +381,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Very important, because cluster is shared between tests and we
 			// don't explicitly check that all slots are available before
 			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 					.addSource(new TestAscendingValueSource(numElements));
@@ -381,15 +391,14 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 					"any",
 					source.getType());
 
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 7662520075515707428L;
+			source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+				private static final long serialVersionUID = 7662520075515707428L;
 
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("hakuna", valueState);
+				@Override
+				public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+					return value.f0;
+				}
+			}).asQueryableState("hakuna", valueState);
 
 			// Submit the job graph
 			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
@@ -397,22 +406,19 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 			cluster.submitJobDetached(jobGraph);
 
-			// Now query
-			long expected = numElements;
-
-			executeQuery(deadline, client, jobId, "hakuna", valueState, expected);
+			executeQuery(deadline, client, jobId, "hakuna", valueState, numElements);
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
 						.getLeaderGateway(deadline.timeLeft())
 						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -425,9 +431,11 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		// Config
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 
-		final int numElements = 1024;
+		final long numElements = 1024L;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -437,7 +445,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Very important, because cluster is shared between tests and we
 			// don't explicitly check that all slots are available before
 			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 				.addSource(new TestAscendingValueSource(numElements));
@@ -481,15 +489,15 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -508,23 +516,25 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		for (int key = 0; key < maxParallelism; key++) {
 			boolean success = false;
 			while (deadline.hasTimeLeft() && !success) {
-				Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
-					jobId,
-					queryableStateName,
-					key,
-					BasicTypeInfo.INT_TYPE_INFO,
-					stateDescriptor,
-					QUERY_RETRY_DELAY,
-					false);
+				CompletableFuture<Tuple2<Integer, Long>> future = getKvStateWithRetries(
+						client,
+						jobId,
+						queryableStateName,
+						key,
+						BasicTypeInfo.INT_TYPE_INFO,
+						stateDescriptor,
+						QUERY_RETRY_DELAY,
+						false,
+						executor);
 
-				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
+				Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 				assertEquals("Key mismatch", key, value.f0.intValue());
 				if (expected == value.f1) {
 					success = true;
 				} else {
 					// Retry
-					Thread.sleep(50);
+					Thread.sleep(50L);
 				}
 			}
 
@@ -554,16 +564,17 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 						BasicTypeInfo.INT_TYPE_INFO,
 						valueSerializer,
 						QUERY_RETRY_DELAY,
-						false);
+						false,
+						executor);
 
-				Tuple2<Integer, Long> value = Await.result(future, deadline.timeLeft());
+				Tuple2<Integer, Long> value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 
 				assertEquals("Key mismatch", key, value.f0.intValue());
 				if (expected == value.f1) {
 					success = true;
 				} else {
 					// Retry
-					Thread.sleep(50);
+					Thread.sleep(50L);
 				}
 			}
 
@@ -575,20 +586,21 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	 * Tests simple value state queryable state instance with a default value
 	 * set. Each source emits (subtaskIndex, 0)..(subtaskIndex, numElements)
 	 * tuples, the key is mapped to 1 but key 0 is queried which should throw
-	 * a {@link UnknownKeyOrNamespace} exception.
+	 * a {@link UnknownKeyOrNamespaceException} exception.
 	 *
-	 * @throws UnknownKeyOrNamespace thrown due querying a non-existent key
+	 * @throws UnknownKeyOrNamespaceException thrown due querying a non-existent key
 	 */
-	@Test(expected = UnknownKeyOrNamespace.class)
-	public void testValueStateDefault() throws
-		Exception, UnknownKeyOrNamespace {
+	@Test(expected = UnknownKeyOrNamespaceException.class)
+	public void testValueStateDefault() throws Throwable {
 
 		// Config
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 
-		final int numElements = 1024;
+		final long numElements = 1024L;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -600,7 +612,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// don't explicitly check that all slots are available before
 			// submitting.
 			env.setRestartStrategy(RestartStrategies
-				.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+				.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 				.addSource(new TestAscendingValueSource(numElements));
@@ -635,30 +647,37 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 			// Now query
 			int key = 0;
-			Future<Tuple2<Integer, Long>> future = getKvStateWithRetries(client,
-				jobId,
-				queryableState.getQueryableStateName(),
-				key,
-				BasicTypeInfo.INT_TYPE_INFO,
-				valueState,
-				QUERY_RETRY_DELAY,
-				true);
-
-			Await.result(future, deadline.timeLeft());
+			CompletableFuture<Tuple2<Integer, Long>> future = getKvStateWithRetries(
+					client,
+					jobId,
+					queryableState.getQueryableStateName(),
+					key,
+					BasicTypeInfo.INT_TYPE_INFO,
+					valueState,
+					QUERY_RETRY_DELAY,
+					true,
+					executor);
+
+			try {
+				future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			} catch (ExecutionException | CompletionException e) {
+				// get() on a completedExceptionally future wraps the
+				// exception in an ExecutionException.
+				throw e.getCause();
+			}
 		} finally {
+
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-					.getLeaderGateway(deadline.timeLeft())
-					.ask(new JobManagerMessages.CancelJob(jobId),
-						deadline.timeLeft())
-					.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(
-						CancellationSuccess.class));
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
+						.getLeaderGateway(deadline.timeLeft())
+						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -675,9 +694,11 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		// Config
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 
-		final int numElements = 1024;
+		final long numElements = 1024L;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -687,7 +708,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Very important, because cluster is shared between tests and we
 			// don't explicitly check that all slots are available before
 			// submitting.
-			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000));
+			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L));
 
 			DataStream<Tuple2<Integer, Long>> source = env
 					.addSource(new TestAscendingValueSource(numElements));
@@ -709,23 +730,21 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 			cluster.submitJobDetached(jobGraph);
 
-			// Now query
-			long expected = numElements;
-
 			executeQuery(deadline, client, jobId, "matata",
-					queryableState.getValueSerializer(), expected);
+					queryableState.getValueSerializer(), numElements);
 		} finally {
+
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
-						.getLeaderGateway(deadline.timeLeft())
-						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(
+						cluster.getLeaderGateway(deadline.timeLeft())
+								.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
+								.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -743,7 +762,9 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 		final int numElements = 1024;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -788,21 +809,23 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			for (int key = 0; key < maxParallelism; key++) {
 				boolean success = false;
 				while (deadline.hasTimeLeft() && !success) {
-					Future<String> future = getKvStateWithRetries(client,
+					CompletableFuture<String> future = getKvStateWithRetries(
+							client,
 							jobId,
 							queryableState.getQueryableStateName(),
 							key,
 							BasicTypeInfo.INT_TYPE_INFO,
 							foldingState,
 							QUERY_RETRY_DELAY,
-							false);
+							false,
+							executor);
 
-					String value = Await.result(future, deadline.timeLeft());
+					String value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 					if (expected.equals(value)) {
 						success = true;
 					} else {
 						// Retry
-						Thread.sleep(50);
+						Thread.sleep(50L);
 					}
 				}
 
@@ -811,15 +834,15 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
 						.getLeaderGateway(deadline.timeLeft())
 						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
@@ -834,9 +857,11 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		// Config
 		final Deadline deadline = TEST_TIMEOUT.fromNow();
 
-		final int numElements = 1024;
+		final long numElements = 1024L;
 
-		final QueryableStateClient client = new QueryableStateClient(cluster.configuration());
+		final QueryableStateClient client = new QueryableStateClient(
+				"localhost",
+				cluster.configuration().getInteger(QueryableStateOptions.SERVER_PORT));
 
 		JobID jobId = null;
 		try {
@@ -858,15 +883,14 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							new SumReduce(),
 							source.getType());
 
-			QueryableStateStream<Integer, Tuple2<Integer, Long>> queryableState =
-					source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
-						private static final long serialVersionUID = 8470749712274833552L;
+			source.keyBy(new KeySelector<Tuple2<Integer, Long>, Integer>() {
+				private static final long serialVersionUID = 8470749712274833552L;
 
-						@Override
-						public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
-							return value.f0;
-						}
-					}).asQueryableState("jungle", reducingState);
+				@Override
+				public Integer getKey(Tuple2<Integer, Long> value) throws Exception {
+					return value.f0;
+				}
+			}).asQueryableState("jungle", reducingState);
 
 			// Submit the job graph
 			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
@@ -877,117 +901,24 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			// Wait until job is running
 
 			// Now query
-			long expected = numElements * (numElements + 1) / 2;
+			long expected = numElements * (numElements + 1L) / 2L;
 
 			executeQuery(deadline, client, jobId, "jungle", reducingState, expected);
 		} finally {
 			// Free cluster resources
 			if (jobId != null) {
-				Future<CancellationSuccess> cancellation = cluster
+				CompletableFuture<CancellationSuccess> cancellation = FutureUtils.toJava(cluster
 						.getLeaderGateway(deadline.timeLeft())
 						.ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft())
-						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class));
+						.mapTo(ClassTag$.MODULE$.<CancellationSuccess>apply(CancellationSuccess.class)));
 
-				Await.ready(cancellation, deadline.timeLeft());
+				cancellation.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
 			}
 
-			client.shutDown();
+			client.shutdown();
 		}
 	}
 
-	private static <K, V> Future<V> getKvStateWithRetries(
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final TypeSerializer<V> valueTypeSerializer,
-			final FiniteDuration retryDelay,
-			final boolean failForUnknownKeyOrNamespace) {
-
-		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, valueTypeSerializer)
-				.recoverWith(new Recover<Future<V>>() {
-					@Override
-					public Future<V> recover(Throwable failure) throws Throwable {
-						if (failure instanceof AssertionError) {
-							return Futures.failed(failure);
-						} else if (failForUnknownKeyOrNamespace &&
-								(failure instanceof UnknownKeyOrNamespace)) {
-							return Futures.failed(failure);
-						} else {
-							// At startup some failures are expected
-							// due to races. Make sure that they don't
-							// fail this test.
-							return Patterns.after(
-									retryDelay,
-									testActorSystem.scheduler(),
-									testActorSystem.dispatcher(),
-									new Callable<Future<V>>() {
-										@Override
-										public Future<V> call() throws Exception {
-											return getKvStateWithRetries(
-													client,
-													jobId,
-													queryName,
-													key,
-													keyTypeInfo,
-													valueTypeSerializer,
-													retryDelay,
-													failForUnknownKeyOrNamespace);
-										}
-									});
-						}
-					}
-				}, testActorSystem.dispatcher());
-
-	}
-
-	private static <K, V> Future<V> getKvStateWithRetries(
-			final QueryableStateClient client,
-			final JobID jobId,
-			final String queryName,
-			final K key,
-			final TypeInformation<K> keyTypeInfo,
-			final StateDescriptor<?, V> stateDescriptor,
-			final FiniteDuration retryDelay,
-			final boolean failForUnknownKeyOrNamespace) {
-
-		return client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor)
-				.recoverWith(new Recover<Future<V>>() {
-					@Override
-					public Future<V> recover(Throwable failure) throws Throwable {
-						if (failure instanceof AssertionError) {
-							return Futures.failed(failure);
-						} else if (failForUnknownKeyOrNamespace &&
-								(failure instanceof UnknownKeyOrNamespace)) {
-							return Futures.failed(failure);
-						} else {
-							// At startup some failures are expected
-							// due to races. Make sure that they don't
-							// fail this test.
-							return Patterns.after(
-									retryDelay,
-									testActorSystem.scheduler(),
-									testActorSystem.dispatcher(),
-									new Callable<Future<V>>() {
-										@Override
-										public Future<V> call() throws Exception {
-											return getKvStateWithRetries(
-													client,
-													jobId,
-													queryName,
-													key,
-													keyTypeInfo,
-													stateDescriptor,
-													retryDelay,
-													failForUnknownKeyOrNamespace);
-										}
-									});
-						}
-					}
-				}, testActorSystem.dispatcher());
-	}
-
 	/**
 	 * Test source producing (key, 0)..(key, maxValue) with key being the sub
 	 * task index.
@@ -1030,7 +961,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 
 			while (isRunning) {
 				synchronized (this) {
-					this.wait();
+					wait();
 				}
 			}
 		}
@@ -1040,7 +971,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			isRunning = false;
 
 			synchronized (this) {
-				this.notifyAll();
+				notifyAll();
 			}
 		}
 
@@ -1125,4 +1056,105 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 		}
 	}
 
+	/////				General Utility Methods				//////
+
+	private static <K, V> Future<V> getKvStateWithRetries(
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final TypeSerializer<V> valueTypeSerializer,
+			final Time retryDelay,
+			final boolean failForUnknownKeyOrNamespace,
+			final ScheduledExecutor executor) {
+
+		return retryWithDelay(
+				() -> client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, valueTypeSerializer),
+				NO_OF_RETRIES,
+				retryDelay,
+				executor,
+				failForUnknownKeyOrNamespace);
+	}
+
+	private static <K, V> CompletableFuture<V> getKvStateWithRetries(
+			final QueryableStateClient client,
+			final JobID jobId,
+			final String queryName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final StateDescriptor<?, V> stateDescriptor,
+			final Time retryDelay,
+			final boolean failForUnknownKeyOrNamespace,
+			final ScheduledExecutor executor) {
+		return retryWithDelay(
+				() -> client.getKvState(jobId, queryName, key, VoidNamespace.INSTANCE, keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor),
+				NO_OF_RETRIES,
+				retryDelay,
+				executor,
+				failForUnknownKeyOrNamespace);
+	}
+
+	private static <T> CompletableFuture<T> retryWithDelay(
+			final Supplier<CompletableFuture<T>> operation,
+			final int retries,
+			final Time retryDelay,
+			final ScheduledExecutor scheduledExecutor,
+			final boolean failIfUnknownKeyOrNamespace) {
+
+		final CompletableFuture<T> resultFuture = new CompletableFuture<>();
+
+		retryWithDelay(
+				resultFuture,
+				operation,
+				retries,
+				retryDelay,
+				scheduledExecutor,
+				failIfUnknownKeyOrNamespace);
+
+		return resultFuture;
+	}
+
+	public static <T> void retryWithDelay(
+			final CompletableFuture<T> resultFuture,
+			final Supplier<CompletableFuture<T>> operation,
+			final int retries,
+			final Time retryDelay,
+			final ScheduledExecutor scheduledExecutor,
+			final boolean failIfUnknownKeyOrNamespace) {
+
+		if (!resultFuture.isDone()) {
+			final CompletableFuture<T> operationResultFuture = operation.get();
+			operationResultFuture.whenCompleteAsync(
+					(t, throwable) -> {
+						if (throwable != null) {
+							if (throwable.getCause() instanceof CancellationException) {
+								resultFuture.completeExceptionally(new FutureUtils.RetryException("Operation future was cancelled.", throwable.getCause()));
+							} else if (throwable.getCause() instanceof AssertionError ||
+									(failIfUnknownKeyOrNamespace && throwable.getCause() instanceof UnknownKeyOrNamespaceException)) {
+								resultFuture.completeExceptionally(throwable.getCause());
+							} else {
+								if (retries > 0) {
+									final ScheduledFuture<?> scheduledFuture = scheduledExecutor.schedule(
+											() -> retryWithDelay(resultFuture, operation, retries - 1, retryDelay, scheduledExecutor, failIfUnknownKeyOrNamespace),
+											retryDelay.toMilliseconds(),
+											TimeUnit.MILLISECONDS);
+
+									resultFuture.whenComplete(
+											(innerT, innerThrowable) -> scheduledFuture.cancel(false));
+								} else {
+									resultFuture.completeExceptionally(new FutureUtils.RetryException("Could not complete the operation. Number of retries " +
+											"has been exhausted.", throwable));
+								}
+							}
+						} else {
+							resultFuture.complete(t);
+						}
+					},
+					scheduledExecutor);
+
+			resultFuture.whenComplete(
+					(t, throwable) -> operationResultFuture.cancel(false));
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
index 15a5ff6..a2a9678 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.QueryableStateOptions;
-import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 
@@ -40,7 +39,7 @@ import static org.junit.Assert.fail;
 public abstract class HAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
 
 	private static final int NUM_JMS = 2;
-	private static final int NUM_TMS = 4;
+	private static final int NUM_TMS = 1;
 	private static final int NUM_SLOTS_PER_TM = 4;
 
 	private static TestingServer zkServer;
@@ -67,8 +66,6 @@ public abstract class HAAbstractQueryableStateITCase extends AbstractQueryableSt
 			cluster = new TestingCluster(config, false);
 			cluster.start();
 
-			testActorSystem = AkkaUtils.createDefaultActorSystem();
-
 			// verify that we are in HA mode
 			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.ZOOKEEPER);
 
@@ -85,9 +82,6 @@ public abstract class HAAbstractQueryableStateITCase extends AbstractQueryableSt
 			cluster.awaitTermination();
 		}
 
-		testActorSystem.shutdown();
-		testActorSystem.awaitTermination();
-
 		try {
 			zkServer.stop();
 			zkServer.close();

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
index c52acc8..1173d0d 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.QueryableStateOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 
@@ -37,7 +36,7 @@ import static org.junit.Assert.fail;
  */
 public abstract class NonHAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
 
-	private static final int NUM_TMS = 2;
+	private static final int NUM_TMS = 1;
 	private static final int NUM_SLOTS_PER_TM = 4;
 
 	@BeforeClass
@@ -48,14 +47,13 @@ public abstract class NonHAAbstractQueryableStateITCase extends AbstractQueryabl
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
 			config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1);
+			config.setInteger(QueryableStateOptions.SERVER_PORT, 9069);
 			config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true);
 			config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1);
 
 			cluster = new TestingCluster(config, false);
 			cluster.start(true);
 
-			testActorSystem = AkkaUtils.createDefaultActorSystem();
-
 			// verify that we are not in HA mode
 			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.NONE);
 
@@ -73,9 +71,5 @@ public abstract class NonHAAbstractQueryableStateITCase extends AbstractQueryabl
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
-
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
deleted file mode 100644
index d9a41a1..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
+++ /dev/null
@@ -1,399 +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.queryablestate.network;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.queryablestate.UnknownJobManager;
-import org.apache.flink.queryablestate.client.AkkaKvStateLocationLookupService;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.akka.FlinkUntypedActor;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
-import org.apache.flink.runtime.query.KvStateLocation;
-import org.apache.flink.runtime.query.KvStateMessage.LookupKvStateLocation;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.Props;
-import akka.actor.Status;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link AkkaKvStateLocationLookupService}.
- */
-public class AkkaKvStateLocationLookupServiceTest extends TestLogger {
-
-	/** The default timeout. */
-	private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
-
-	/** Test actor system shared between the tests. */
-	private static ActorSystem testActorSystem;
-
-	@BeforeClass
-	public static void setUp() throws Exception {
-		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-	}
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
-	}
-
-	/**
-	 * Tests responses if no leader notification has been reported or leadership
-	 * has been lost (leaderAddress = <code>null</code>).
-	 */
-	@Test
-	public void testNoJobManagerRegistered() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			null,
-			null);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		//
-		// No leader registered initially => fail with UnknownJobManager
-		//
-		try {
-			JobID jobId = new JobID();
-			String name = "coffee";
-
-			Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(jobId, name);
-
-			Await.result(locationFuture, TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (UnknownJobManager ignored) {
-			// Expected
-		}
-
-		assertEquals("Received unexpected lookup", 0, received.size());
-
-		//
-		// Leader registration => communicate with new leader
-		//
-		UUID leaderSessionId = HighAvailabilityServices.DEFAULT_LEADER_ID;
-		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "tea");
-
-		ActorRef testActor = LookupResponseActor.create(received, leaderSessionId, expected);
-
-		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		// Notify the service about a leader
-		leaderRetrievalService.notifyListener(testActorAddress, leaderSessionId);
-
-		JobID jobId = new JobID();
-		String name = "tea";
-
-		// Verify that the leader response is handled
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, name), TIMEOUT);
-		assertEquals(expected, location);
-
-		// Verify that the correct message was sent to the leader
-		assertEquals(1, received.size());
-
-		verifyLookupMsg(received.poll(), jobId, name);
-
-		//
-		// Leader loss => fail with UnknownJobManager
-		//
-		leaderRetrievalService.notifyListener(null, null);
-
-		try {
-			Future<KvStateLocation> locationFuture = lookupService
-					.getKvStateLookupInfo(new JobID(), "coffee");
-
-			Await.result(locationFuture, TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (UnknownJobManager ignored) {
-			// Expected
-		}
-
-		// No new messages received
-		assertEquals(0, received.size());
-	}
-
-	/**
-	 * Tests that messages are properly decorated with the leader session ID.
-	 */
-	@Test
-	public void testLeaderSessionIdChange() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			"localhost",
-			HighAvailabilityServices.DEFAULT_LEADER_ID);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		// Create test actors with random leader session IDs
-		KvStateLocation expected1 = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "salt");
-		UUID leaderSessionId1 = UUID.randomUUID();
-		ActorRef testActor1 = LookupResponseActor.create(received, leaderSessionId1, expected1);
-		String testActorAddress1 = AkkaUtils.getAkkaURL(testActorSystem, testActor1);
-
-		KvStateLocation expected2 = new KvStateLocation(new JobID(), new JobVertexID(), 22321, "pepper");
-		UUID leaderSessionId2 = UUID.randomUUID();
-		ActorRef testActor2 = LookupResponseActor.create(received, leaderSessionId1, expected2);
-		String testActorAddress2 = AkkaUtils.getAkkaURL(testActorSystem, testActor2);
-
-		JobID jobId = new JobID();
-
-		//
-		// Notify about first leader
-		//
-		leaderRetrievalService.notifyListener(testActorAddress1, leaderSessionId1);
-
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, "rock"), TIMEOUT);
-		assertEquals(expected1, location);
-
-		assertEquals(1, received.size());
-		verifyLookupMsg(received.poll(), jobId, "rock");
-
-		//
-		// Notify about second leader
-		//
-		leaderRetrievalService.notifyListener(testActorAddress2, leaderSessionId2);
-
-		location = Await.result(lookupService.getKvStateLookupInfo(jobId, "roll"), TIMEOUT);
-		assertEquals(expected2, location);
-
-		assertEquals(1, received.size());
-		verifyLookupMsg(received.poll(), jobId, "roll");
-	}
-
-	/**
-	 * Tests that lookups are retried when no leader notification is available.
-	 */
-	@Test
-	public void testRetryOnUnknownJobManager() throws Exception {
-		final Queue<AkkaKvStateLocationLookupService.LookupRetryStrategy> retryStrategies = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService.LookupRetryStrategyFactory retryStrategy =
-				new AkkaKvStateLocationLookupService.LookupRetryStrategyFactory() {
-					@Override
-					public AkkaKvStateLocationLookupService.LookupRetryStrategy createRetryStrategy() {
-						return retryStrategies.poll();
-					}
-				};
-
-		final TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			null,
-			null);
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				retryStrategy);
-
-		lookupService.start();
-
-		//
-		// Test call to retry
-		//
-		final AtomicBoolean hasRetried = new AtomicBoolean();
-		retryStrategies.add(
-				new AkkaKvStateLocationLookupService.LookupRetryStrategy() {
-					@Override
-					public FiniteDuration getRetryDelay() {
-						return FiniteDuration.Zero();
-					}
-
-					@Override
-					public boolean tryRetry() {
-						if (hasRetried.compareAndSet(false, true)) {
-							return true;
-						}
-						return false;
-					}
-				});
-
-		Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(new JobID(), "yessir");
-
-		Await.ready(locationFuture, TIMEOUT);
-		assertTrue("Did not retry ", hasRetried.get());
-
-		//
-		// Test leader notification after retry
-		//
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 12122, "garlic");
-		ActorRef testActor = LookupResponseActor.create(received, null, expected);
-		final String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		retryStrategies.add(new AkkaKvStateLocationLookupService.LookupRetryStrategy() {
-			@Override
-			public FiniteDuration getRetryDelay() {
-				return FiniteDuration.apply(100, TimeUnit.MILLISECONDS);
-			}
-
-			@Override
-			public boolean tryRetry() {
-				leaderRetrievalService.notifyListener(testActorAddress, HighAvailabilityServices.DEFAULT_LEADER_ID);
-				return true;
-			}
-		});
-
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(new JobID(), "yessir"), TIMEOUT);
-		assertEquals(expected, location);
-	}
-
-	@Test
-	public void testUnexpectedResponseType() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			"localhost",
-			HighAvailabilityServices.DEFAULT_LEADER_ID);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		// Create test actors with random leader session IDs
-		String expected = "unexpected-response-type";
-		ActorRef testActor = LookupResponseActor.create(received, null, expected);
-		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		leaderRetrievalService.notifyListener(testActorAddress, null);
-
-		try {
-			Await.result(lookupService.getKvStateLookupInfo(new JobID(), "spicy"), TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (Throwable ignored) {
-			// Expected
-		}
-	}
-
-	private static final class LookupResponseActor extends FlinkUntypedActor {
-
-		/** Received lookup messages. */
-		private final Queue<LookupKvStateLocation> receivedLookups;
-
-		/** Responses on KvStateMessage.LookupKvStateLocation messages. */
-		private final Queue<Object> lookupResponses;
-
-		/** The leader session ID. */
-		private UUID leaderSessionId;
-
-		public LookupResponseActor(
-				Queue<LookupKvStateLocation> receivedLookups,
-				UUID leaderSessionId, Object... lookupResponses) {
-
-			this.receivedLookups = Preconditions.checkNotNull(receivedLookups, "Received lookups");
-			this.leaderSessionId = leaderSessionId;
-			this.lookupResponses = new ArrayDeque<>();
-
-			if (lookupResponses != null) {
-				for (Object resp : lookupResponses) {
-					this.lookupResponses.add(resp);
-				}
-			}
-		}
-
-		@Override
-		public void handleMessage(Object message) throws Exception {
-			if (message instanceof LookupKvStateLocation) {
-				// Add to received lookups queue
-				receivedLookups.add((LookupKvStateLocation) message);
-
-				Object msg = lookupResponses.poll();
-				if (msg != null) {
-					if (msg instanceof Throwable) {
-						sender().tell(new Status.Failure((Throwable) msg), self());
-					} else {
-						sender().tell(new Status.Success(msg), self());
-					}
-				}
-			} else if (message instanceof UUID) {
-				this.leaderSessionId = (UUID) message;
-			} else {
-				LOG.debug("Received unhandled message: {}", message);
-			}
-		}
-
-		@Override
-		protected UUID getLeaderSessionID() {
-			return leaderSessionId;
-		}
-
-		private static ActorRef create(
-				Queue<LookupKvStateLocation> receivedLookups,
-				UUID leaderSessionId,
-				Object... lookupResponses) {
-
-			return testActorSystem.actorOf(Props.create(
-					LookupResponseActor.class,
-					receivedLookups,
-					leaderSessionId,
-					lookupResponses));
-		}
-	}
-
-	private static void verifyLookupMsg(
-			LookupKvStateLocation lookUpMsg,
-			JobID expectedJobId,
-			String expectedName) {
-
-		assertNotNull(lookUpMsg);
-		assertEquals(expectedJobId, lookUpMsg.getJobId());
-		assertEquals(expectedName, lookUpMsg.getRegistrationName());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
new file mode 100644
index 0000000..b6f855e
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
@@ -0,0 +1,784 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+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.InternalKvState;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link Client}.
+ */
+public class ClientTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ClientTest.class);
+
+	// Thread pool for client bootstrap (shared between tests)
+	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
+
+	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10L, TimeUnit.SECONDS);
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (NIO_GROUP != null) {
+			NIO_GROUP.shutdownGracefully();
+		}
+	}
+
+	/**
+	 * Tests simple queries, of which half succeed and half fail.
+	 */
+	@Test
+	public void testSimpleRequests() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new Client<>("Test Client", 1, serializer, stats);
+
+			// Random result
+			final byte[] expected = new byte[1024];
+			ThreadLocalRandom.current().nextBytes(expected);
+
+			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.add((ByteBuf) msg);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			long numQueries = 1024L;
+
+			List<CompletableFuture<KvStateResponse>> futures = new ArrayList<>();
+			for (long i = 0L; i < numQueries; i++) {
+				KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+				futures.add(client.sendRequest(serverAddress, request));
+			}
+
+			// Respond to messages
+			Exception testException = new RuntimeException("Expected test Exception");
+
+			for (long i = 0L; i < numQueries; i++) {
+				ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				assertNotNull("Receive timed out", buf);
+
+				Channel ch = channel.get();
+				assertNotNull("Channel not active", ch);
+
+				assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+				long requestId = MessageSerializer.getRequestId(buf);
+				KvStateInternalRequest deserRequest = serializer.deserializeRequest(buf);
+
+				buf.release();
+
+				if (i % 2L == 0L) {
+					ByteBuf response = MessageSerializer.serializeResponse(
+							serverChannel.alloc(),
+							requestId,
+							new KvStateResponse(expected));
+
+					ch.writeAndFlush(response);
+				} else {
+					ByteBuf response = MessageSerializer.serializeRequestFailure(
+							serverChannel.alloc(),
+							requestId,
+							testException);
+
+					ch.writeAndFlush(response);
+				}
+			}
+
+			for (long i = 0L; i < numQueries; i++) {
+
+				if (i % 2L == 0L) {
+					KvStateResponse serializedResult = futures.get((int) i).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+					assertArrayEquals(expected, serializedResult.getContent());
+				} else {
+					try {
+						futures.get((int) i).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+						fail("Did not throw expected Exception");
+					} catch (ExecutionException e) {
+
+						if (!(e.getCause() instanceof RuntimeException)) {
+							fail("Did not throw expected Exception");
+						}
+						// else expected
+					}
+				}
+			}
+
+			assertEquals(numQueries, stats.getNumRequests());
+			long expectedRequests = numQueries / 2L;
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != expectedRequests ||
+					stats.getNumFailed() != expectedRequests)) {
+				Thread.sleep(100L);
+			}
+
+			assertEquals(expectedRequests, stats.getNumSuccessful());
+			assertEquals(expectedRequests, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutdown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0L, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a request to an unavailable host is failed with ConnectException.
+	 */
+	@Test
+	public void testRequestUnavailableHost() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+
+		try {
+			client = new Client<>("Test Client", 1, serializer, stats);
+
+			int availablePort = NetUtils.getAvailablePort();
+
+			KvStateServerAddress serverAddress = new KvStateServerAddress(
+					InetAddress.getLocalHost(),
+					availablePort);
+
+			KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+			CompletableFuture<KvStateResponse> future = client.sendRequest(serverAddress, request);
+
+			try {
+				future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				fail("Did not throw expected ConnectException");
+			} catch (ExecutionException e) {
+				if (!(e.getCause() instanceof ConnectException)) {
+					fail("Did not throw expected ConnectException");
+				}
+				// else expected
+			}
+		} finally {
+			if (client != null) {
+				client.shutdown();
+			}
+
+			assertEquals("Channel leak", 0L, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Multiple threads concurrently fire queries.
+	 */
+	@Test
+	public void testConcurrentQueries() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		ExecutorService executor = null;
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+		Channel serverChannel = null;
+
+		final byte[] serializedResult = new byte[1024];
+		ThreadLocalRandom.current().nextBytes(serializedResult);
+
+		try {
+			int numQueryTasks = 4;
+			final int numQueriesPerTask = 1024;
+
+			executor = Executors.newFixedThreadPool(numQueryTasks);
+
+			client = new Client<>("Test Client", 1, serializer, stats);
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					ByteBuf buf = (ByteBuf) msg;
+					assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+					long requestId = MessageSerializer.getRequestId(buf);
+					KvStateInternalRequest request = serializer.deserializeRequest(buf);
+
+					buf.release();
+
+					KvStateResponse response = new KvStateResponse(serializedResult);
+					ByteBuf serResponse = MessageSerializer.serializeResponse(
+							ctx.alloc(),
+							requestId,
+							response);
+
+					ctx.channel().writeAndFlush(serResponse);
+				}
+			});
+
+			final KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			final Client<KvStateInternalRequest, KvStateResponse> finalClient = client;
+			Callable<List<CompletableFuture<KvStateResponse>>> queryTask = () -> {
+				List<CompletableFuture<KvStateResponse>> results = new ArrayList<>(numQueriesPerTask);
+
+				for (int i = 0; i < numQueriesPerTask; i++) {
+					KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+					results.add(finalClient.sendRequest(serverAddress, request));
+				}
+
+				return results;
+			};
+
+			// Submit query tasks
+			List<Future<List<CompletableFuture<KvStateResponse>>>> futures = new ArrayList<>();
+			for (int i = 0; i < numQueryTasks; i++) {
+				futures.add(executor.submit(queryTask));
+			}
+
+			// Verify results
+			for (Future<List<CompletableFuture<KvStateResponse>>> future : futures) {
+				List<CompletableFuture<KvStateResponse>> results = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				for (CompletableFuture<KvStateResponse> result : results) {
+					KvStateResponse actual = result.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+					assertArrayEquals(serializedResult, actual.getContent());
+				}
+			}
+
+			int totalQueries = numQueryTasks * numQueriesPerTask;
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && stats.getNumSuccessful() != totalQueries) {
+				Thread.sleep(100L);
+			}
+
+			assertEquals(totalQueries, stats.getNumRequests());
+			assertEquals(totalQueries, stats.getNumSuccessful());
+		} finally {
+			if (executor != null) {
+				executor.shutdown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			if (client != null) {
+				client.shutdown();
+			}
+
+			assertEquals("Channel leak", 0L, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a server failure closes the connection and removes it from
+	 * the established connections.
+	 */
+	@Test
+	public void testFailureClosesChannel() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new Client<>("Test Client", 1, serializer, stats);
+
+			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.add((ByteBuf) msg);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			// Requests
+			List<Future<KvStateResponse>> futures = new ArrayList<>();
+			KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+
+			futures.add(client.sendRequest(serverAddress, request));
+			futures.add(client.sendRequest(serverAddress, request));
+
+			ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			assertNotNull("Receive timed out", buf);
+			buf.release();
+
+			buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			assertNotNull("Receive timed out", buf);
+			buf.release();
+
+			assertEquals(1L, stats.getNumConnections());
+
+			Channel ch = channel.get();
+			assertNotNull("Channel not active", ch);
+
+			// Respond with failure
+			ch.writeAndFlush(MessageSerializer.serializeServerFailure(
+					serverChannel.alloc(),
+					new RuntimeException("Expected test server failure")));
+
+			try {
+				futures.remove(0).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				fail("Did not throw expected server failure");
+			} catch (ExecutionException e) {
+
+				if (!(e.getCause() instanceof RuntimeException)) {
+					fail("Did not throw expected Exception");
+				}
+				// Expected
+			}
+
+			try {
+				futures.remove(0).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				fail("Did not throw expected server failure");
+			} catch (ExecutionException e) {
+
+				if (!(e.getCause() instanceof RuntimeException)) {
+					fail("Did not throw expected Exception");
+				}
+				// Expected
+			}
+
+			assertEquals(0L, stats.getNumConnections());
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0L || stats.getNumFailed() != 2L)) {
+				Thread.sleep(100L);
+			}
+
+			assertEquals(2L, stats.getNumRequests());
+			assertEquals(0L, stats.getNumSuccessful());
+			assertEquals(2L, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutdown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0L, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a server channel close, closes the connection and removes it
+	 * from the established connections.
+	 */
+	@Test
+	public void testServerClosesChannel() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new Client<>("Test Client", 1, serializer, stats);
+
+			final AtomicBoolean received = new AtomicBoolean();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.set(true);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			// Requests
+			KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+			Future<KvStateResponse> future = client.sendRequest(serverAddress, request);
+
+			while (!received.get() && deadline.hasTimeLeft()) {
+				Thread.sleep(50L);
+			}
+			assertTrue("Receive timed out", received.get());
+
+			assertEquals(1, stats.getNumConnections());
+
+			channel.get().close().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+
+			try {
+				future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				fail("Did not throw expected server failure");
+			} catch (ExecutionException e) {
+				if (!(e.getCause() instanceof ClosedChannelException)) {
+					fail("Did not throw expected Exception");
+				}
+				// Expected
+			}
+
+			assertEquals(0L, stats.getNumConnections());
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0L || stats.getNumFailed() != 1L)) {
+				Thread.sleep(100L);
+			}
+
+			assertEquals(1L, stats.getNumRequests());
+			assertEquals(0L, stats.getNumSuccessful());
+			assertEquals(1L, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutdown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0L, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests multiple clients querying multiple servers until 100k queries have
+	 * been processed. At this point, the client is shut down and its verified
+	 * that all ongoing requests are failed.
+	 */
+	@Test
+	public void testClientServerIntegration() throws Exception {
+		// Config
+		final int numServers = 2;
+		final int numServerEventLoopThreads = 2;
+		final int numServerQueryThreads = 2;
+
+		final int numClientEventLoopThreads = 4;
+		final int numClientsTasks = 8;
+
+		final int batchSize = 16;
+
+		final int numKeyGroups = 1;
+
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		KvStateRegistry dummyRegistry = new KvStateRegistry();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(dummyRegistry);
+
+		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				dummyRegistry.createTaskRegistry(new JobID(), new JobVertexID()));
+
+		final FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+
+		AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats();
+
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		Client<KvStateInternalRequest, KvStateResponse> client = null;
+		ExecutorService clientTaskExecutor = null;
+		final KvStateServerImpl[] server = new KvStateServerImpl[numServers];
+
+		try {
+			client = new Client<>("Test Client", numClientEventLoopThreads, serializer, clientStats);
+			clientTaskExecutor = Executors.newFixedThreadPool(numClientsTasks);
+
+			// Create state
+			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+			desc.setQueryable("any");
+
+			// Create servers
+			KvStateRegistry[] registry = new KvStateRegistry[numServers];
+			AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
+			final KvStateID[] ids = new KvStateID[numServers];
+
+			for (int i = 0; i < numServers; i++) {
+				registry[i] = new KvStateRegistry();
+				serverStats[i] = new AtomicKvStateRequestStats();
+				server[i] = new KvStateServerImpl(
+						InetAddress.getLocalHost(),
+						0,
+						numServerEventLoopThreads,
+						numServerQueryThreads,
+						registry[i],
+						serverStats[i]);
+
+				server[i].start();
+
+				backend.setCurrentKey(1010 + i);
+
+				// Value per server
+				ValueState<Integer> state = backend.getPartitionedState(VoidNamespace.INSTANCE,
+						VoidNamespaceSerializer.INSTANCE,
+						desc);
+
+				state.update(201 + i);
+
+				// we know it must be a KvStat but this is not exposed to the user via 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);
+			}
+
+			final Client<KvStateInternalRequest, KvStateResponse> finalClient = client;
+			Callable<Void> queryTask = () -> {
+				while (true) {
+					if (Thread.interrupted()) {
+						throw new InterruptedException();
+					}
+
+					// Random server permutation
+					List<Integer> random = new ArrayList<>();
+					for (int j = 0; j < batchSize; j++) {
+						random.add(j);
+					}
+					Collections.shuffle(random);
+
+					// Dispatch queries
+					List<Future<KvStateResponse>> futures = new ArrayList<>(batchSize);
+
+					for (int j = 0; j < batchSize; j++) {
+						int targetServer = random.get(j) % numServers;
+
+						byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+								1010 + targetServer,
+								IntSerializer.INSTANCE,
+								VoidNamespace.INSTANCE,
+								VoidNamespaceSerializer.INSTANCE);
+
+						KvStateInternalRequest request = new KvStateInternalRequest(ids[targetServer], serializedKeyAndNamespace);
+						futures.add(finalClient.sendRequest(server[targetServer].getServerAddress(), request));
+					}
+
+					// Verify results
+					for (int j = 0; j < batchSize; j++) {
+						int targetServer = random.get(j) % numServers;
+
+						Future<KvStateResponse> future = futures.get(j);
+						byte[] buf = future.get(timeout.toMillis(), TimeUnit.MILLISECONDS).getContent();
+						int value = KvStateSerializer.deserializeValue(buf, IntSerializer.INSTANCE);
+						assertEquals(201L + targetServer, value);
+					}
+				}
+			};
+
+			// Submit tasks
+			List<Future<Void>> taskFutures = new ArrayList<>();
+			for (int i = 0; i < numClientsTasks; i++) {
+				taskFutures.add(clientTaskExecutor.submit(queryTask));
+			}
+
+			long numRequests;
+			while ((numRequests = clientStats.getNumRequests()) < 100_000L) {
+				Thread.sleep(100L);
+				LOG.info("Number of requests {}/100_000", numRequests);
+			}
+
+			// Shut down
+			client.shutdown();
+
+			for (Future<Void> future : taskFutures) {
+				try {
+					future.get();
+					fail("Did not throw expected Exception after shut down");
+				} catch (ExecutionException t) {
+					if (t.getCause().getCause() instanceof ClosedChannelException ||
+							t.getCause().getCause() instanceof IllegalStateException) {
+						// Expected
+					} else {
+						t.printStackTrace();
+						fail("Failed with unexpected Exception type: " + t.getClass().getName());
+					}
+				}
+			}
+
+			assertEquals("Connection leak (client)", 0L, clientStats.getNumConnections());
+			for (int i = 0; i < numServers; i++) {
+				boolean success = false;
+				int numRetries = 0;
+				while (!success) {
+					try {
+						assertEquals("Connection leak (server)", 0L, serverStats[i].getNumConnections());
+						success = true;
+					} catch (Throwable t) {
+						if (numRetries < 10) {
+							LOG.info("Retrying connection leak check (server)");
+							Thread.sleep((numRetries + 1) * 50L);
+							numRetries++;
+						} else {
+							throw t;
+						}
+					}
+				}
+			}
+		} finally {
+			if (client != null) {
+				client.shutdown();
+			}
+
+			for (int i = 0; i < numServers; i++) {
+				if (server[i] != null) {
+					server[i].shutdown();
+				}
+			}
+
+			if (clientTaskExecutor != null) {
+				clientTaskExecutor.shutdown();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private Channel createServerChannel(final ChannelHandler... handlers) throws UnknownHostException, InterruptedException {
+		ServerBootstrap bootstrap = new ServerBootstrap()
+				// Bind address and port
+				.localAddress(InetAddress.getLocalHost(), 0)
+				// NIO server channels
+				.group(NIO_GROUP)
+				.channel(NioServerSocketChannel.class)
+				// See initializer for pipeline details
+				.childHandler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					protected void initChannel(SocketChannel ch) throws Exception {
+						ch.pipeline()
+								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+								.addLast(handlers);
+					}
+				});
+
+		return bootstrap.bind().sync().channel();
+	}
+
+	private KvStateServerAddress getKvStateServerAddress(Channel serverChannel) {
+		InetSocketAddress localAddress = (InetSocketAddress) serverChannel.localAddress();
+
+		return new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
index 0b97bda..cb490aa 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.queryablestate.network;
 
-import org.apache.flink.queryablestate.client.KvStateClientHandler;
-import org.apache.flink.queryablestate.client.KvStateClientHandlerCallback;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
 import org.apache.flink.queryablestate.network.messages.MessageSerializer;
 
 import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
@@ -37,7 +37,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 /**
- * Tests for {@link KvStateClientHandler}.
+ * Tests for {@link ClientHandler}.
  */
 public class KvStateClientHandlerTest {
 
@@ -47,28 +47,30 @@ public class KvStateClientHandlerTest {
 	 */
 	@Test
 	public void testReadCallbacksAndBufferRecycling() throws Exception {
-		KvStateClientHandlerCallback callback = mock(KvStateClientHandlerCallback.class);
+		final ClientHandlerCallback<KvStateResponse> callback = mock(ClientHandlerCallback.class);
 
-		EmbeddedChannel channel = new EmbeddedChannel(new KvStateClientHandler(callback));
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+		final EmbeddedChannel channel = new EmbeddedChannel(new ClientHandler<>("Test Client", serializer, callback));
+
+		final byte[] content = new byte[0];
+		final KvStateResponse response = new KvStateResponse(content);
 
 		//
 		// Request success
 		//
-		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
-				channel.alloc(),
-				1222112277,
-				new byte[0]);
+		ByteBuf buf = MessageSerializer.serializeResponse(channel.alloc(), 1222112277L, response);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify callback
 		channel.writeInbound(buf);
-		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(byte[].class));
+		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(KvStateResponse.class));
 		assertEquals("Buffer not recycled", 0, buf.refCnt());
 
 		//
 		// Request failure
 		//
-		buf = MessageSerializer.serializeKvStateRequestFailure(
+		buf = MessageSerializer.serializeRequestFailure(
 				channel.alloc(),
 				1222112278,
 				new RuntimeException("Expected test Exception"));


[12/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
new file mode 100644
index 0000000..e6d59de
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/Client.java
@@ -0,0 +1,537 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayDeque;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The base class for every client in the queryable state module.
+ * It is using pure netty to send and receive messages of type {@link MessageBody}.
+ *
+ * @param <REQ> the type of request the client will send.
+ * @param <RESP> the type of response the client expects to receive.
+ */
+@Internal
+public class Client<REQ extends MessageBody, RESP extends MessageBody> {
+
+	/** The name of the client. Used for logging and stack traces.*/
+	private final String clientName;
+
+	/** Netty's Bootstrap. */
+	private final Bootstrap bootstrap;
+
+	/** The serializer to be used for (de-)serializing messages. */
+	private final MessageSerializer<REQ, RESP> messageSerializer;
+
+	/** Statistics tracker. */
+	private final KvStateRequestStats stats;
+
+	/** Established connections. */
+	private final Map<KvStateServerAddress, EstablishedConnection> establishedConnections = new ConcurrentHashMap<>();
+
+	/** Pending connections. */
+	private final Map<KvStateServerAddress, PendingConnection> pendingConnections = new ConcurrentHashMap<>();
+
+	/** Atomic shut down flag. */
+	private final AtomicBoolean shutDown = new AtomicBoolean();
+
+	/**
+	 * Creates a client with the specified number of event loop threads.
+	 *
+	 * @param clientName the name of the client.
+	 * @param numEventLoopThreads number of event loop threads (minimum 1).
+	 * @param serializer the serializer used to (de-)serialize messages.
+	 * @param stats the statistics collector.
+	 */
+	public Client(
+			final String clientName,
+			final int numEventLoopThreads,
+			final MessageSerializer<REQ, RESP> serializer,
+			final KvStateRequestStats stats) {
+
+		Preconditions.checkArgument(numEventLoopThreads >= 1,
+				"Non-positive number of event loop threads.");
+
+		this.clientName = Preconditions.checkNotNull(clientName);
+		this.messageSerializer = Preconditions.checkNotNull(serializer);
+		this.stats = Preconditions.checkNotNull(stats);
+
+		final ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink " + clientName + " Event Loop Thread %d")
+				.build();
+
+		final EventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
+		final ByteBufAllocator bufferPool = new NettyBufferPool(numEventLoopThreads);
+
+		this.bootstrap = new Bootstrap()
+				.group(nioGroup)
+				.channel(NioSocketChannel.class)
+				.option(ChannelOption.ALLOCATOR, bufferPool)
+				.handler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					protected void initChannel(SocketChannel channel) throws Exception {
+						channel.pipeline()
+								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+								.addLast(new ChunkedWriteHandler());
+					}
+				});
+	}
+
+	public String getClientName() {
+		return clientName;
+	}
+
+	public CompletableFuture<RESP> sendRequest(final KvStateServerAddress serverAddress, final REQ request) {
+		if (shutDown.get()) {
+			return FutureUtils.getFailedFuture(new IllegalStateException("Shut down"));
+		}
+
+		EstablishedConnection connection = establishedConnections.get(serverAddress);
+		if (connection != null) {
+			return connection.sendRequest(request);
+		} else {
+			PendingConnection pendingConnection = pendingConnections.get(serverAddress);
+			if (pendingConnection != null) {
+				// There was a race, use the existing pending connection.
+				return pendingConnection.sendRequest(request);
+			} else {
+				// We try to connect to the server.
+				PendingConnection pending = new PendingConnection(serverAddress, messageSerializer);
+				PendingConnection previous = pendingConnections.putIfAbsent(serverAddress, pending);
+
+				if (previous == null) {
+					// OK, we are responsible to connect.
+					bootstrap.connect(serverAddress.getHost(), serverAddress.getPort()).addListener(pending);
+					return pending.sendRequest(request);
+				} else {
+					// There was a race, use the existing pending connection.
+					return previous.sendRequest(request);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Shuts down the client and closes all connections.
+	 *
+	 * <p>After a call to this method, all returned futures will be failed.
+	 */
+	public void shutdown() {
+		if (shutDown.compareAndSet(false, true)) {
+			for (Map.Entry<KvStateServerAddress, EstablishedConnection> conn : establishedConnections.entrySet()) {
+				if (establishedConnections.remove(conn.getKey(), conn.getValue())) {
+					conn.getValue().close();
+				}
+			}
+
+			for (Map.Entry<KvStateServerAddress, PendingConnection> conn : pendingConnections.entrySet()) {
+				if (pendingConnections.remove(conn.getKey()) != null) {
+					conn.getValue().close();
+				}
+			}
+
+			if (bootstrap != null) {
+				EventLoopGroup group = bootstrap.group();
+				if (group != null) {
+					group.shutdownGracefully(0L, 10L, TimeUnit.SECONDS);
+				}
+			}
+		}
+	}
+
+	/**
+	 * A pending connection that is in the process of connecting.
+	 */
+	private class PendingConnection implements ChannelFutureListener {
+
+		/** Lock to guard the connect call, channel hand in, etc. */
+		private final Object connectLock = new Object();
+
+		/** Address of the server we are connecting to. */
+		private final KvStateServerAddress serverAddress;
+
+		private final MessageSerializer<REQ, RESP> serializer;
+
+		/** Queue of requests while connecting. */
+		private final ArrayDeque<PendingRequest> queuedRequests = new ArrayDeque<>();
+
+		/** The established connection after the connect succeeds. */
+		private EstablishedConnection established;
+
+		/** Closed flag. */
+		private boolean closed;
+
+		/** Failure cause if something goes wrong. */
+		private Throwable failureCause;
+
+		/**
+		 * Creates a pending connection to the given server.
+		 *
+		 * @param serverAddress Address of the server to connect to.
+		 */
+		private PendingConnection(
+				final KvStateServerAddress serverAddress,
+				final MessageSerializer<REQ, RESP> serializer) {
+			this.serverAddress = serverAddress;
+			this.serializer = serializer;
+		}
+
+		@Override
+		public void operationComplete(ChannelFuture future) throws Exception {
+			if (future.isSuccess()) {
+				handInChannel(future.channel());
+			} else {
+				close(future.cause());
+			}
+		}
+
+		/**
+		 * Returns a future holding the serialized request result.
+		 *
+		 * <p>If the channel has been established, forward the call to the
+		 * established channel, otherwise queue it for when the channel is
+		 * handed in.
+		 *
+		 * @param request the request to be sent.
+		 * @return Future holding the serialized result
+		 */
+		public CompletableFuture<RESP> sendRequest(REQ request) {
+			synchronized (connectLock) {
+				if (failureCause != null) {
+					return FutureUtils.getFailedFuture(failureCause);
+				} else if (closed) {
+					return FutureUtils.getFailedFuture(new ClosedChannelException());
+				} else {
+					if (established != null) {
+						return established.sendRequest(request);
+					} else {
+						// Queue this and handle when connected
+						final PendingRequest pending = new PendingRequest(request);
+						queuedRequests.add(pending);
+						return pending;
+					}
+				}
+			}
+		}
+
+		/**
+		 * Hands in a channel after a successful connection.
+		 *
+		 * @param channel Channel to hand in
+		 */
+		private void handInChannel(Channel channel) {
+			synchronized (connectLock) {
+				if (closed || failureCause != null) {
+					// Close the channel and we are done. Any queued requests
+					// are removed on the close/failure call and after that no
+					// new ones can be enqueued.
+					channel.close();
+				} else {
+					established = new EstablishedConnection(serverAddress, serializer, channel);
+
+					while (!queuedRequests.isEmpty()) {
+						final PendingRequest pending = queuedRequests.poll();
+
+						established.sendRequest(pending.request)
+								.thenAccept(resp -> pending.complete(resp))
+								.exceptionally(throwable -> {
+									pending.completeExceptionally(throwable);
+									return null;
+						});
+					}
+
+					// Publish the channel for the general public
+					establishedConnections.put(serverAddress, established);
+					pendingConnections.remove(serverAddress);
+
+					// Check shut down for possible race with shut down. We
+					// don't want any lingering connections after shut down,
+					// which can happen if we don't check this here.
+					if (shutDown.get()) {
+						if (establishedConnections.remove(serverAddress, established)) {
+							established.close();
+						}
+					}
+				}
+			}
+		}
+
+		/**
+		 * Close the connecting channel with a ClosedChannelException.
+		 */
+		private void close() {
+			close(new ClosedChannelException());
+		}
+
+		/**
+		 * Close the connecting channel with an Exception (can be {@code null})
+		 * or forward to the established channel.
+		 */
+		private void close(Throwable cause) {
+			synchronized (connectLock) {
+				if (!closed) {
+					if (failureCause == null) {
+						failureCause = cause;
+					}
+
+					if (established != null) {
+						established.close();
+					} else {
+						PendingRequest pending;
+						while ((pending = queuedRequests.poll()) != null) {
+							pending.completeExceptionally(cause);
+						}
+					}
+					closed = true;
+				}
+			}
+		}
+
+		@Override
+		public String toString() {
+			synchronized (connectLock) {
+				return "PendingConnection{" +
+						"serverAddress=" + serverAddress +
+						", queuedRequests=" + queuedRequests.size() +
+						", established=" + (established != null) +
+						", closed=" + closed +
+						'}';
+			}
+		}
+
+		/**
+		 * A pending request queued while the channel is connecting.
+		 */
+		private final class PendingRequest extends CompletableFuture<RESP> {
+
+			private final REQ request;
+
+			private PendingRequest(REQ request) {
+				this.request = request;
+			}
+		}
+	}
+
+	/**
+	 * An established connection that wraps the actual channel instance and is
+	 * registered at the {@link ClientHandler} for callbacks.
+	 */
+	private class EstablishedConnection implements ClientHandlerCallback<RESP> {
+
+		/** Address of the server we are connected to. */
+		private final KvStateServerAddress serverAddress;
+
+		/** The actual TCP channel. */
+		private final Channel channel;
+
+		/** Pending requests keyed by request ID. */
+		private final ConcurrentHashMap<Long, TimestampedCompletableFuture> pendingRequests = new ConcurrentHashMap<>();
+
+		/** Current request number used to assign unique request IDs. */
+		private final AtomicLong requestCount = new AtomicLong();
+
+		/** Reference to a failure that was reported by the channel. */
+		private final AtomicReference<Throwable> failureCause = new AtomicReference<>();
+
+		/**
+		 * Creates an established connection with the given channel.
+		 *
+		 * @param serverAddress Address of the server connected to
+		 * @param channel The actual TCP channel
+		 */
+		EstablishedConnection(
+				final KvStateServerAddress serverAddress,
+				final MessageSerializer<REQ, RESP> serializer,
+				final Channel channel) {
+
+			this.serverAddress = Preconditions.checkNotNull(serverAddress);
+			this.channel = Preconditions.checkNotNull(channel);
+
+			// Add the client handler with the callback
+			channel.pipeline().addLast(
+					getClientName() + " Handler",
+					new ClientHandler<>(clientName, serializer, this)
+			);
+
+			stats.reportActiveConnection();
+		}
+
+		/**
+		 * Close the channel with a ClosedChannelException.
+		 */
+		void close() {
+			close(new ClosedChannelException());
+		}
+
+		/**
+		 * Close the channel with a cause.
+		 *
+		 * @param cause The cause to close the channel with.
+		 * @return Channel close future
+		 */
+		private boolean close(Throwable cause) {
+			if (failureCause.compareAndSet(null, cause)) {
+				channel.close();
+				stats.reportInactiveConnection();
+
+				for (long requestId : pendingRequests.keySet()) {
+					TimestampedCompletableFuture pending = pendingRequests.remove(requestId);
+					if (pending != null && pending.completeExceptionally(cause)) {
+						stats.reportFailedRequest();
+					}
+				}
+				return true;
+			}
+			return false;
+		}
+
+		/**
+		 * Returns a future holding the serialized request result.
+		 * @param request the request to be sent.
+		 * @return Future holding the serialized result
+		 */
+		CompletableFuture<RESP> sendRequest(REQ request) {
+			TimestampedCompletableFuture requestPromiseTs =
+					new TimestampedCompletableFuture(System.nanoTime());
+			try {
+				final long requestId = requestCount.getAndIncrement();
+				pendingRequests.put(requestId, requestPromiseTs);
+
+				stats.reportRequest();
+
+				ByteBuf buf = MessageSerializer.serializeRequest(channel.alloc(), requestId, request);
+
+				channel.writeAndFlush(buf).addListener((ChannelFutureListener) future -> {
+					if (!future.isSuccess()) {
+						// Fail promise if not failed to write
+						TimestampedCompletableFuture pending = pendingRequests.remove(requestId);
+						if (pending != null && pending.completeExceptionally(future.cause())) {
+							stats.reportFailedRequest();
+						}
+					}
+				});
+
+				// Check failure for possible race. We don't want any lingering
+				// promises after a failure, which can happen if we don't check
+				// this here. Note that close is treated as a failure as well.
+				Throwable failure = failureCause.get();
+				if (failure != null) {
+					// Remove from pending requests to guard against concurrent
+					// removal and to make sure that we only count it once as failed.
+					TimestampedCompletableFuture pending = pendingRequests.remove(requestId);
+					if (pending != null && pending.completeExceptionally(failure)) {
+						stats.reportFailedRequest();
+					}
+				}
+			} catch (Throwable t) {
+				requestPromiseTs.completeExceptionally(t);
+			}
+
+			return requestPromiseTs;
+		}
+
+		@Override
+		public void onRequestResult(long requestId, RESP response) {
+			TimestampedCompletableFuture pending = pendingRequests.remove(requestId);
+			if (pending != null && pending.complete(response)) {
+				long durationMillis = (System.nanoTime() - pending.getTimestamp()) / 1_000_000L;
+				stats.reportSuccessfulRequest(durationMillis);
+			}
+		}
+
+		@Override
+		public void onRequestFailure(long requestId, Throwable cause) {
+			TimestampedCompletableFuture pending = pendingRequests.remove(requestId);
+			if (pending != null && pending.completeExceptionally(cause)) {
+				stats.reportFailedRequest();
+			}
+		}
+
+		@Override
+		public void onFailure(Throwable cause) {
+			if (close(cause)) {
+				// Remove from established channels, otherwise future
+				// requests will be handled by this failed channel.
+				establishedConnections.remove(serverAddress, this);
+			}
+		}
+
+		@Override
+		public String toString() {
+			return "EstablishedConnection{" +
+					"serverAddress=" + serverAddress +
+					", channel=" + channel +
+					", pendingRequests=" + pendingRequests.size() +
+					", requestCount=" + requestCount +
+					", failureCause=" + failureCause +
+					'}';
+		}
+
+		/**
+		 * Pair of promise and a timestamp.
+		 */
+		private class TimestampedCompletableFuture extends CompletableFuture<RESP> {
+
+			private final long timestampInNanos;
+
+			TimestampedCompletableFuture(long timestampInNanos) {
+				this.timestampInNanos = timestampInNanos;
+			}
+
+			public long getTimestamp() {
+				return timestampInNanos;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java
new file mode 100644
index 0000000..fc9b1d4
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandler.java
@@ -0,0 +1,122 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.network.messages.RequestFailure;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.channels.ClosedChannelException;
+
+/**
+ * The handler used by a {@link Client} to handling incoming messages.
+ *
+ * @param <REQ> the type of request the client will send.
+ * @param <RESP> the type of response the client expects to receive.
+ */
+@Internal
+public class ClientHandler<REQ extends MessageBody, RESP extends MessageBody> extends ChannelInboundHandlerAdapter {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ClientHandler.class);
+
+	private final String clientName;
+
+	private final MessageSerializer<REQ, RESP> serializer;
+
+	private final ClientHandlerCallback<RESP> callback;
+
+	/**
+	 * Creates a handler with the callback.
+	 *
+	 * @param clientName the name of the client.
+	 * @param serializer the serializer used to (de-)serialize messages.
+	 * @param callback Callback for responses.
+	 */
+	public ClientHandler(
+			final String clientName,
+			final MessageSerializer<REQ, RESP> serializer,
+			final ClientHandlerCallback<RESP> callback) {
+
+		this.clientName = Preconditions.checkNotNull(clientName);
+		this.serializer = Preconditions.checkNotNull(serializer);
+		this.callback = Preconditions.checkNotNull(callback);
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		try {
+			ByteBuf buf = (ByteBuf) msg;
+			MessageType msgType = MessageSerializer.deserializeHeader(buf);
+
+			if (msgType == MessageType.REQUEST_RESULT) {
+				long requestId = MessageSerializer.getRequestId(buf);
+				RESP result = serializer.deserializeResponse(buf);
+				callback.onRequestResult(requestId, result);
+			} else if (msgType == MessageType.REQUEST_FAILURE) {
+				RequestFailure failure = MessageSerializer.deserializeRequestFailure(buf);
+				callback.onRequestFailure(failure.getRequestId(), failure.getCause());
+			} else if (msgType == MessageType.SERVER_FAILURE) {
+				throw MessageSerializer.deserializeServerFailure(buf);
+			} else {
+				throw new IllegalStateException("Unexpected response type '" + msgType + "'");
+			}
+		} catch (Throwable t1) {
+			try {
+				callback.onFailure(t1);
+			} catch (Throwable t2) {
+				LOG.error("Failed to notify callback about failure", t2);
+			}
+		} finally {
+			ReferenceCountUtil.release(msg);
+		}
+	}
+
+	@Override
+	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+		try {
+			callback.onFailure(cause);
+		} catch (Throwable t) {
+			LOG.error("Failed to notify callback about failure", t);
+		}
+	}
+
+	@Override
+	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+		// Only the client is expected to close the channel. Otherwise it
+		// indicates a failure. Note that this will be invoked in both cases
+		// though. If the callback closed the channel, the callback must be
+		// ignored.
+		try {
+			callback.onFailure(new ClosedChannelException());
+		} catch (Throwable t) {
+			LOG.error("Failed to notify callback about failure", t);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.java
new file mode 100644
index 0000000..00ce1ed
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/ClientHandlerCallback.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.queryablestate.network;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.network.messages.MessageBody;
+
+/**
+ * Callback for {@link ClientHandler}.
+ */
+@Internal
+public interface ClientHandlerCallback<RESP extends MessageBody> {
+
+	/**
+	 * Called on a successful request.
+	 *
+	 * @param requestId			ID of the request
+	 * @param response			The received response
+	 */
+	void onRequestResult(long requestId, RESP response);
+
+	/**
+	 * Called on a failed request.
+	 *
+	 * @param requestId ID of the request
+	 * @param cause     Cause of the request failure
+	 */
+	void onRequestFailure(long requestId, Throwable cause);
+
+	/**
+	 * Called on any failure, which is not related to a specific request.
+	 *
+	 * <p>This can be for example a caught Exception in the channel pipeline
+	 * or an unexpected channel close.
+	 *
+	 * @param cause Cause of the failure
+	 */
+	void onFailure(Throwable cause);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java
new file mode 100644
index 0000000..f26c267
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageBody.java
@@ -0,0 +1,38 @@
+/*
+ * 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.queryablestate.network.messages;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * The base class for every message exchanged during the communication between
+ * {@link org.apache.flink.queryablestate.network.Client client} and
+ * {@link org.apache.flink.queryablestate.network.AbstractServerBase server}.
+ *
+ * <p>Every such message should also have a {@link MessageDeserializer}.
+ */
+@Internal
+public abstract class MessageBody {
+
+	/**
+	 * Serializes the message into a byte array.
+	 * @return A byte array with the serialized content of the message.
+	 */
+	public abstract byte[] serialize();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java
new file mode 100644
index 0000000..436fb82
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageDeserializer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.network.messages;
+
+import org.apache.flink.annotation.Internal;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+/**
+ * A utility used to deserialize a {@link MessageBody message}.
+ * @param <M> The type of the message to be deserialized.
+ *           It has to extend {@link MessageBody}
+ */
+@Internal
+public interface MessageDeserializer<M extends MessageBody> {
+
+	/**
+	 * Deserializes a message contained in a byte buffer.
+	 * @param buf the buffer containing the message.
+	 * @return The deserialized message.
+	 */
+	M deserializeMessage(ByteBuf buf);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
index 32bca64..c0a0d32 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageSerializer.java
@@ -18,11 +18,7 @@
 
 package org.apache.flink.queryablestate.network.messages;
 
-import org.apache.flink.queryablestate.messages.KvStateRequest;
-import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
-import org.apache.flink.queryablestate.messages.KvStateRequestResult;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.util.AbstractID;
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.util.Preconditions;
 
 import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
@@ -37,8 +33,8 @@ import java.io.ObjectOutputStream;
 
 /**
  * Serialization and deserialization of messages exchanged between
- * {@link org.apache.flink.queryablestate.client.KvStateClient client} and
- * {@link org.apache.flink.queryablestate.server.KvStateServerImpl server}.
+ * {@link org.apache.flink.queryablestate.network.Client client} and
+ * {@link org.apache.flink.queryablestate.network.AbstractServerBase server}.
  *
  * <p>The binary messages have the following format:
  *
@@ -52,8 +48,12 @@ import java.io.ObjectOutputStream;
  * </pre>
  *
  * <p>The concrete content of a message depends on the {@link MessageType}.
+ *
+ * @param <REQ>		Type of the requests of the protocol.
+ * @param <RESP>	Type of the responses of the protocol.
  */
-public final class MessageSerializer {
+@Internal
+public final class MessageSerializer<REQ extends MessageBody, RESP extends MessageBody> {
 
 	/** The serialization version ID. */
 	private static final int VERSION = 0x79a1b710;
@@ -64,78 +64,58 @@ public final class MessageSerializer {
 	/** Byte length of the request id. */
 	private static final int REQUEST_ID_SIZE = Long.BYTES;
 
+	/** The constructor of the {@link MessageBody client requests}. Used for deserialization. */
+	private final MessageDeserializer<REQ> requestDeserializer;
+
+	/** The constructor of the {@link MessageBody server responses}. Used for deserialization. */
+	private final MessageDeserializer<RESP> responseDeserializer;
+
+	public MessageSerializer(MessageDeserializer<REQ> requestDeser, MessageDeserializer<RESP> responseDeser) {
+		requestDeserializer = Preconditions.checkNotNull(requestDeser);
+		responseDeserializer = Preconditions.checkNotNull(responseDeser);
+	}
+
 	// ------------------------------------------------------------------------
 	// Serialization
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Allocates a buffer and serializes the KvState request into it.
+	 * Serializes the request sent to the
+	 * {@link org.apache.flink.queryablestate.network.AbstractServerBase}.
 	 *
-	 * @param alloc                     ByteBuf allocator for the buffer to
-	 *                                  serialize message into
-	 * @param requestId                 ID for this request
-	 * @param kvStateId                 ID of the requested KvState instance
-	 * @param serializedKeyAndNamespace Serialized key and namespace to request
-	 *                                  from the KvState instance.
-	 * @return Serialized KvState request message
+	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
+	 * @param requestId		The id of the request to which the message refers to.
+	 * @param request		The request to be serialized.
+	 * @return A {@link ByteBuf} containing the serialized message.
 	 */
-	public static ByteBuf serializeKvStateRequest(
-			ByteBufAllocator alloc,
-			long requestId,
-			KvStateID kvStateId,
-			byte[] serializedKeyAndNamespace) {
-
-		// Header + request ID + KvState ID + Serialized namespace
-		int frameLength = HEADER_LENGTH + REQUEST_ID_SIZE + AbstractID.SIZE + (Integer.BYTES + serializedKeyAndNamespace.length);
-		ByteBuf buf = alloc.ioBuffer(frameLength + 4); // +4 for frame length
-
-		buf.writeInt(frameLength);
-
-		writeHeader(buf, MessageType.REQUEST);
-
-		buf.writeLong(requestId);
-		buf.writeLong(kvStateId.getLowerPart());
-		buf.writeLong(kvStateId.getUpperPart());
-		buf.writeInt(serializedKeyAndNamespace.length);
-		buf.writeBytes(serializedKeyAndNamespace);
-
-		return buf;
+	public static <REQ extends MessageBody> ByteBuf serializeRequest(
+			final ByteBufAllocator alloc,
+			final long requestId,
+			final REQ request) {
+		Preconditions.checkNotNull(request);
+		return writePayload(alloc, requestId, MessageType.REQUEST, request.serialize());
 	}
 
 	/**
-	 * Allocates a buffer and serializes the KvState request result into it.
+	 * Serializes the response sent to the
+	 * {@link org.apache.flink.queryablestate.network.Client}.
 	 *
-	 * @param alloc             ByteBuf allocator for the buffer to serialize message into
-	 * @param requestId         ID for this request
-	 * @param serializedResult  Serialized Result
-	 * @return Serialized KvState request result message
+	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
+	 * @param requestId		The id of the request to which the message refers to.
+	 * @param response		The response to be serialized.
+	 * @return A {@link ByteBuf} containing the serialized message.
 	 */
-	public static ByteBuf serializeKvStateRequestResult(
-			ByteBufAllocator alloc,
-			long requestId,
-			byte[] serializedResult) {
-
-		Preconditions.checkNotNull(serializedResult, "Serialized result");
-
-		// Header + request ID + serialized result
-		int frameLength = HEADER_LENGTH + REQUEST_ID_SIZE + 4 + serializedResult.length;
-
-		// TODO: 10/5/17 there was a bug all this time?
-		ByteBuf buf = alloc.ioBuffer(frameLength + 4);
-
-		buf.writeInt(frameLength);
-		writeHeader(buf, MessageType.REQUEST_RESULT);
-		buf.writeLong(requestId);
-
-		buf.writeInt(serializedResult.length);
-		buf.writeBytes(serializedResult);
-
-		return buf;
+	public static <RESP extends MessageBody> ByteBuf serializeResponse(
+			final ByteBufAllocator alloc,
+			final long requestId,
+			final RESP response) {
+		Preconditions.checkNotNull(response);
+		return writePayload(alloc, requestId, MessageType.REQUEST_RESULT, response.serialize());
 	}
 
 	/**
 	 * Serializes the exception containing the failure message sent to the
-	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * {@link org.apache.flink.queryablestate.network.Client} in case of
 	 * protocol related errors.
 	 *
 	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
@@ -143,7 +123,7 @@ public final class MessageSerializer {
 	 * @param cause			The exception thrown at the server.
 	 * @return A {@link ByteBuf} containing the serialized message.
 	 */
-	public static ByteBuf serializeKvStateRequestFailure(
+	public static ByteBuf serializeRequestFailure(
 			final ByteBufAllocator alloc,
 			final long requestId,
 			final Throwable cause) throws IOException {
@@ -168,7 +148,7 @@ public final class MessageSerializer {
 
 	/**
 	 * Serializes the failure message sent to the
-	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * {@link org.apache.flink.queryablestate.network.Client} in case of
 	 * server related errors.
 	 *
 	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
@@ -207,6 +187,31 @@ public final class MessageSerializer {
 		buf.writeInt(messageType.ordinal());
 	}
 
+	/**
+	 * Helper for serializing the messages.
+	 *
+	 * @param alloc			The {@link ByteBufAllocator} used to allocate the buffer to serialize the message into.
+	 * @param requestId		The id of the request to which the message refers to.
+	 * @param messageType	The {@link MessageType type of the message}.
+	 * @param payload		The serialized version of the message.
+	 * @return A {@link ByteBuf} containing the serialized message.
+	 */
+	private static ByteBuf writePayload(
+			final ByteBufAllocator alloc,
+			final long requestId,
+			final MessageType messageType,
+			final byte[] payload) {
+
+		final int frameLength = HEADER_LENGTH + REQUEST_ID_SIZE + payload.length;
+		final ByteBuf buf = alloc.ioBuffer(frameLength + Integer.BYTES);
+
+		buf.writeInt(frameLength);
+		writeHeader(buf, messageType);
+		buf.writeLong(requestId);
+		buf.writeBytes(payload);
+		return buf;
+	}
+
 	// ------------------------------------------------------------------------
 	// Deserialization
 	// ------------------------------------------------------------------------
@@ -230,71 +235,54 @@ public final class MessageSerializer {
 		// fetching the message type
 		int msgType = buf.readInt();
 		MessageType[] values = MessageType.values();
-		Preconditions.checkState(msgType >= 0 && msgType <= values.length,
+		Preconditions.checkState(msgType >= 0 && msgType < values.length,
 				"Illegal message type with index " + msgType + '.');
 		return values[msgType];
 	}
 
 	/**
-	 * Deserializes the KvState request message.
-	 *
-	 * <p><strong>Important</strong>: the returned buffer is sliced from the
-	 * incoming ByteBuf stream and retained. Therefore, it needs to be recycled
-	 * by the consumer.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequest
+	 * De-serializes the header and returns the {@link MessageType}.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the request id position.</b>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized request id.
+	 * @return		The request id.
 	 */
-	public static KvStateRequest deserializeKvStateRequest(ByteBuf buf) {
-		long requestId = buf.readLong();
-		KvStateID kvStateId = new KvStateID(buf.readLong(), buf.readLong());
-
-		// Serialized key and namespace
-		int length = buf.readInt();
-
-		if (length < 0) {
-			throw new IllegalArgumentException("Negative length for serialized key and namespace. " +
-					"This indicates a serialization error.");
-		}
-
-		// Copy the buffer in order to be able to safely recycle the ByteBuf
-		byte[] serializedKeyAndNamespace = new byte[length];
-		if (length > 0) {
-			buf.readBytes(serializedKeyAndNamespace);
-		}
-
-		return new KvStateRequest(requestId, kvStateId, serializedKeyAndNamespace);
+	public static long getRequestId(final ByteBuf buf) {
+		return buf.readLong();
 	}
 
 	/**
-	 * Deserializes the KvState request result.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequestResult
+	 * De-serializes the request sent to the
+	 * {@link org.apache.flink.queryablestate.network.AbstractServerBase}.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the request position.</b>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized request.
+	 * @return		The request.
 	 */
-	public static KvStateRequestResult deserializeKvStateRequestResult(ByteBuf buf) {
-		long requestId = buf.readLong();
-
-		// Serialized KvState
-		int length = buf.readInt();
-
-		if (length < 0) {
-			throw new IllegalArgumentException("Negative length for serialized result. " +
-					"This indicates a serialization error.");
-		}
-
-		byte[] serializedValue = new byte[length];
-
-		if (length > 0) {
-			buf.readBytes(serializedValue);
-		}
+	public REQ deserializeRequest(final ByteBuf buf) {
+		Preconditions.checkNotNull(buf);
+		return requestDeserializer.deserializeMessage(buf);
+	}
 
-		return new KvStateRequestResult(requestId, serializedValue);
+	/**
+	 * De-serializes the response sent to the
+	 * {@link org.apache.flink.queryablestate.network.Client}.
+	 * <pre>
+	 *  <b>The buffer is expected to be at the response position.</b>
+	 * </pre>
+	 * @param buf	The {@link ByteBuf} containing the serialized response.
+	 * @return		The response.
+	 */
+	public RESP deserializeResponse(final ByteBuf buf) {
+		Preconditions.checkNotNull(buf);
+		return responseDeserializer.deserializeMessage(buf);
 	}
 
 	/**
-	 * De-serializes the {@link KvStateRequestFailure} sent to the
-	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * De-serializes the {@link RequestFailure} sent to the
+	 * {@link org.apache.flink.queryablestate.network.Client} in case of
 	 * protocol related errors.
 	 * <pre>
 	 *  <b>The buffer is expected to be at the correct position.</b>
@@ -302,7 +290,7 @@ public final class MessageSerializer {
 	 * @param buf	The {@link ByteBuf} containing the serialized failure message.
 	 * @return		The failure message.
 	 */
-	public static KvStateRequestFailure deserializeKvStateRequestFailure(final ByteBuf buf) throws IOException, ClassNotFoundException {
+	public static RequestFailure deserializeRequestFailure(final ByteBuf buf) throws IOException, ClassNotFoundException {
 		long requestId = buf.readLong();
 
 		Throwable cause;
@@ -310,12 +298,12 @@ public final class MessageSerializer {
 				ObjectInputStream in = new ObjectInputStream(bis)) {
 			cause = (Throwable) in.readObject();
 		}
-		return new KvStateRequestFailure(requestId, cause);
+		return new RequestFailure(requestId, cause);
 	}
 
 	/**
 	 * De-serializes the failure message sent to the
-	 * {@link org.apache.flink.queryablestate.client.KvStateClient} in case of
+	 * {@link org.apache.flink.queryablestate.network.Client} in case of
 	 * server related errors.
 	 * <pre>
 	 *  <b>The buffer is expected to be at the correct position.</b>

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
index 4e4435d..562ce93 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/MessageType.java
@@ -18,11 +18,14 @@
 
 package org.apache.flink.queryablestate.network.messages;
 
+import org.apache.flink.annotation.Internal;
+
 /**
  * Expected message types during the communication between
- * {@link org.apache.flink.queryablestate.client.KvStateClient state client} and
- * {@link org.apache.flink.queryablestate.server.KvStateServerImpl state server}.
+ * {@link org.apache.flink.queryablestate.network.Client client} and
+ * {@link org.apache.flink.queryablestate.network.AbstractServerBase server}.
  */
+@Internal
 public enum MessageType {
 
 	/** The message is a request. */

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java
new file mode 100644
index 0000000..106199f
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/network/messages/RequestFailure.java
@@ -0,0 +1,71 @@
+/*
+ * 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.queryablestate.network.messages;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A message indicating a protocol-related error.
+ */
+@Internal
+public class RequestFailure {
+
+	/** ID of the request responding to. */
+	private final long requestId;
+
+	/** Failure cause. Not allowed to be a user type. */
+	private final Throwable cause;
+
+	/**
+	 * Creates a failure response to a {@link MessageBody}.
+	 *
+	 * @param requestId ID for the request responding to
+	 * @param cause     Failure cause (not allowed to be a user type)
+	 */
+	public RequestFailure(long requestId, Throwable cause) {
+		this.requestId = requestId;
+		this.cause = cause;
+	}
+
+	/**
+	 * Returns the request ID responding to.
+	 *
+	 * @return Request ID responding to
+	 */
+	public long getRequestId() {
+		return requestId;
+	}
+
+	/**
+	 * Returns the failure cause.
+	 *
+	 * @return Failure cause
+	 */
+	public Throwable getCause() {
+		return cause;
+	}
+
+	@Override
+	public String toString() {
+		return "RequestFailure{" +
+				"requestId=" + requestId +
+				", cause=" + cause +
+				'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
deleted file mode 100644
index f10969e..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/ChunkedByteBuf.java
+++ /dev/null
@@ -1,98 +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.queryablestate.server;
-
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedInput;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
-/**
- * A {@link ByteBuf} instance to be consumed in chunks by {@link ChunkedWriteHandler},
- * respecting the high and low watermarks.
- *
- * @see <a href="http://normanmaurer.me/presentations/2014-facebook-eng-netty/slides.html#10.0">Low/High Watermarks</a>
- */
-public class ChunkedByteBuf implements ChunkedInput<ByteBuf> {
-
-	/** The buffer to chunk. */
-	private final ByteBuf buf;
-
-	/** Size of chunks. */
-	private final int chunkSize;
-
-	/** Closed flag. */
-	private boolean isClosed;
-
-	/** End of input flag. */
-	private boolean isEndOfInput;
-
-	public ChunkedByteBuf(ByteBuf buf, int chunkSize) {
-		this.buf = Preconditions.checkNotNull(buf, "Buffer");
-		Preconditions.checkArgument(chunkSize > 0, "Non-positive chunk size");
-		this.chunkSize = chunkSize;
-	}
-
-	@Override
-	public boolean isEndOfInput() throws Exception {
-		return isClosed || isEndOfInput;
-	}
-
-	@Override
-	public void close() throws Exception {
-		if (!isClosed) {
-			// If we did not consume the whole buffer yet, we have to release
-			// it here. Otherwise, it's the responsibility of the consumer.
-			if (!isEndOfInput) {
-				buf.release();
-			}
-
-			isClosed = true;
-		}
-	}
-
-	@Override
-	public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception {
-		if (isClosed) {
-			return null;
-		} else if (buf.readableBytes() <= chunkSize) {
-			isEndOfInput = true;
-
-			// Don't retain as the consumer is responsible to release it
-			return buf.slice();
-		} else {
-			// Return a chunk sized slice of the buffer. The ref count is
-			// shared with the original buffer. That's why we need to retain
-			// a reference here.
-			return buf.readSlice(chunkSize).retain();
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "ChunkedByteBuf{" +
-				"buf=" + buf +
-				", chunkSize=" + chunkSize +
-				", isClosed=" + isClosed +
-				", isEndOfInput=" + isEndOfInput +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
index 9a31fca..055a5d0 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
@@ -18,31 +18,25 @@
 
 package org.apache.flink.queryablestate.server;
 
-import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
-import org.apache.flink.queryablestate.UnknownKvStateID;
-import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespaceException;
+import org.apache.flink.queryablestate.UnknownKvStateIdException;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.AbstractServerHandler;
 import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.queryablestate.network.messages.MessageType;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.netty.KvStateRequestStats;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.Preconditions;
 
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
 import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.Objects;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.CompletableFuture;
 
 /**
  * This handler dispatches asynchronous tasks, which query {@link InternalKvState}
@@ -52,257 +46,62 @@ import java.util.concurrent.TimeUnit;
  * query task. The actual query is handled in a separate thread as it might
  * otherwise block the network threads (file I/O etc.).
  */
+@Internal
 @ChannelHandler.Sharable
-public class KvStateServerHandler extends ChannelInboundHandlerAdapter {
+public class KvStateServerHandler extends AbstractServerHandler<KvStateInternalRequest, KvStateResponse> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerHandler.class);
 
 	/** KvState registry holding references to the KvState instances. */
 	private final KvStateRegistry registry;
 
-	/** Thread pool for query execution. */
-	private final ExecutorService queryExecutor;
-
-	/** Exposed server statistics. */
-	private final KvStateRequestStats stats;
-
 	/**
-	 * Create the handler.
+	 * Create the handler used by the {@link KvStateServerImpl}.
 	 *
-	 * @param kvStateRegistry Registry to query.
-	 * @param queryExecutor   Thread pool for query execution.
-	 * @param stats           Exposed server statistics.
+	 * @param server the {@link KvStateServerImpl} using the handler.
+	 * @param kvStateRegistry registry to query.
+	 * @param serializer the {@link MessageSerializer} used to (de-) serialize the different messages.
+	 * @param stats server statistics collector.
 	 */
 	public KvStateServerHandler(
-			KvStateRegistry kvStateRegistry,
-			ExecutorService queryExecutor,
-			KvStateRequestStats stats) {
+			final KvStateServerImpl server,
+			final KvStateRegistry kvStateRegistry,
+			final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer,
+			final KvStateRequestStats stats) {
 
-		this.registry = Objects.requireNonNull(kvStateRegistry, "KvStateRegistry");
-		this.queryExecutor = Objects.requireNonNull(queryExecutor, "Query thread pool");
-		this.stats = Objects.requireNonNull(stats, "KvStateRequestStats");
+		super(server, serializer, stats);
+		this.registry = Preconditions.checkNotNull(kvStateRegistry);
 	}
 
 	@Override
-	public void channelActive(ChannelHandlerContext ctx) throws Exception {
-		stats.reportActiveConnection();
-	}
-
-	@Override
-	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
-		stats.reportInactiveConnection();
-	}
-
-	@Override
-	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-		KvStateRequest request = null;
+	public CompletableFuture<KvStateResponse> handleRequest(final long requestId, final KvStateInternalRequest request) {
+		final CompletableFuture<KvStateResponse> responseFuture = new CompletableFuture<>();
 
 		try {
-			ByteBuf buf = (ByteBuf) msg;
-			MessageType msgType = MessageSerializer.deserializeHeader(buf);
-
-			if (msgType == MessageType.REQUEST) {
-				// ------------------------------------------------------------
-				// Request
-				// ------------------------------------------------------------
-				request = MessageSerializer.deserializeKvStateRequest(buf);
-
-				stats.reportRequest();
-
-				InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
-
-				if (kvState != null) {
-					// Execute actual query async, because it is possibly
-					// blocking (e.g. file I/O).
-					//
-					// A submission failure is not treated as fatal.
-					queryExecutor.submit(new AsyncKvStateQueryTask(ctx, request, kvState, stats));
-				} else {
-					ByteBuf unknown = MessageSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(),
-							request.getRequestId(),
-							new UnknownKvStateID(request.getKvStateId()));
-
-					ctx.writeAndFlush(unknown);
-
-					stats.reportFailedRequest();
-				}
+			final InternalKvState<?> kvState = registry.getKvState(request.getKvStateId());
+			if (kvState == null) {
+				responseFuture.completeExceptionally(new UnknownKvStateIdException(getServerName(), request.getKvStateId()));
 			} else {
-				// ------------------------------------------------------------
-				// Unexpected
-				// ------------------------------------------------------------
-				ByteBuf failure = MessageSerializer.serializeServerFailure(
-						ctx.alloc(),
-						new IllegalArgumentException("Unexpected message type " + msgType
-								+ ". KvStateServerHandler expects "
-								+ MessageType.REQUEST + " messages."));
-
-				ctx.writeAndFlush(failure);
-			}
-		} catch (Throwable t) {
-			String stringifiedCause = ExceptionUtils.stringifyException(t);
-
-			ByteBuf err;
-			if (request != null) {
-				String errMsg = "Failed to handle incoming request with ID " +
-						request.getRequestId() + ". Caused by: " + stringifiedCause;
-				err = MessageSerializer.serializeKvStateRequestFailure(
-						ctx.alloc(),
-						request.getRequestId(),
-						new RuntimeException(errMsg));
-
-				stats.reportFailedRequest();
-			} else {
-				String errMsg = "Failed to handle incoming message. Caused by: " + stringifiedCause;
-				err = MessageSerializer.serializeServerFailure(
-						ctx.alloc(),
-						new RuntimeException(errMsg));
-			}
-
-			ctx.writeAndFlush(err);
-		} finally {
-			// IMPORTANT: We have to always recycle the incoming buffer.
-			// Otherwise we will leak memory out of Netty's buffer pool.
-			//
-			// If any operation ever holds on to the buffer, it is the
-			// responsibility of that operation to retain the buffer and
-			// release it later.
-			ReferenceCountUtil.release(msg);
-		}
-	}
-
-	@Override
-	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-		String stringifiedCause = ExceptionUtils.stringifyException(cause);
-		String msg = "Exception in server pipeline. Caused by: " + stringifiedCause;
-
-		ByteBuf err = MessageSerializer.serializeServerFailure(
-				ctx.alloc(),
-				new RuntimeException(msg));
-
-		ctx.writeAndFlush(err).addListener(ChannelFutureListener.CLOSE);
-	}
-
-	/**
-	 * Task to execute the actual query against the {@link InternalKvState} instance.
-	 */
-	private static class AsyncKvStateQueryTask implements Runnable {
-
-		private final ChannelHandlerContext ctx;
-
-		private final KvStateRequest request;
-
-		private final InternalKvState<?> kvState;
-
-		private final KvStateRequestStats stats;
-
-		private final long creationNanos;
-
-		public AsyncKvStateQueryTask(
-				ChannelHandlerContext ctx,
-				KvStateRequest request,
-				InternalKvState<?> kvState,
-				KvStateRequestStats stats) {
-
-			this.ctx = Objects.requireNonNull(ctx, "Channel handler context");
-			this.request = Objects.requireNonNull(request, "State query");
-			this.kvState = Objects.requireNonNull(kvState, "KvState");
-			this.stats = Objects.requireNonNull(stats, "State query stats");
-			this.creationNanos = System.nanoTime();
-		}
-
-		@Override
-		public void run() {
-			boolean success = false;
-
-			try {
-				if (!ctx.channel().isActive()) {
-					return;
-				}
-
-				// Query the KvState instance
 				byte[] serializedKeyAndNamespace = request.getSerializedKeyAndNamespace();
-				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
 
+				byte[] serializedResult = kvState.getSerializedValue(serializedKeyAndNamespace);
 				if (serializedResult != null) {
-					// We found some data, success!
-					ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
-							ctx.alloc(),
-							request.getRequestId(),
-							serializedResult);
-
-					int highWatermark = ctx.channel().config().getWriteBufferHighWaterMark();
-
-					ChannelFuture write;
-					if (buf.readableBytes() <= highWatermark) {
-						write = ctx.writeAndFlush(buf);
-					} else {
-						write = ctx.writeAndFlush(new ChunkedByteBuf(buf, highWatermark));
-					}
-
-					write.addListener(new QueryResultWriteListener());
-
-					success = true;
+					responseFuture.complete(new KvStateResponse(serializedResult));
 				} else {
-					// No data for the key/namespace. This is considered to be
-					// a failure.
-					ByteBuf unknownKey = MessageSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(),
-							request.getRequestId(),
-							new UnknownKeyOrNamespace());
-
-					ctx.writeAndFlush(unknownKey);
-				}
-			} catch (Throwable t) {
-				try {
-					String stringifiedCause = ExceptionUtils.stringifyException(t);
-					String errMsg = "Failed to query state backend for query " +
-							request.getRequestId() + ". Caused by: " + stringifiedCause;
-
-					ByteBuf err = MessageSerializer.serializeKvStateRequestFailure(
-							ctx.alloc(), request.getRequestId(), new RuntimeException(errMsg));
-
-					ctx.writeAndFlush(err);
-				} catch (IOException e) {
-					LOG.error("Failed to respond with the error after failed to query state backend", e);
-				}
-			} finally {
-				if (!success) {
-					stats.reportFailedRequest();
+					responseFuture.completeExceptionally(new UnknownKeyOrNamespaceException(getServerName()));
 				}
 			}
+			return responseFuture;
+		} catch (Throwable t) {
+			String errMsg = "Error while processing request with ID " + requestId +
+					". Caused by: " + ExceptionUtils.stringifyException(t);
+			responseFuture.completeExceptionally(new RuntimeException(errMsg));
+			return responseFuture;
 		}
+	}
 
-		@Override
-		public String toString() {
-			return "AsyncKvStateQueryTask{" +
-					", request=" + request +
-					", creationNanos=" + creationNanos +
-					'}';
-		}
-
-		/**
-		 * Callback after query result has been written.
-		 *
-		 * <p>Gathers stats and logs errors.
-		 */
-		private class QueryResultWriteListener implements ChannelFutureListener {
-
-			@Override
-			public void operationComplete(ChannelFuture future) throws Exception {
-				long durationNanos = System.nanoTime() - creationNanos;
-				long durationMillis = TimeUnit.MILLISECONDS.convert(durationNanos, TimeUnit.NANOSECONDS);
-
-				if (future.isSuccess()) {
-					stats.reportSuccessfulRequest(durationMillis);
-				} else {
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Query " + request + " failed after " + durationMillis + " ms", future.cause());
-					}
-
-					stats.reportFailedRequest();
-				}
-			}
-		}
+	@Override
+	public void shutdown() {
+		// do nothing
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
index 4bf7e24..b4c548a 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java
@@ -18,213 +18,93 @@
 
 package org.apache.flink.queryablestate.server;
 
-import org.apache.flink.queryablestate.messages.KvStateRequest;
-import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.AbstractServerBase;
+import org.apache.flink.queryablestate.network.AbstractServerHandler;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.KvStateServer;
 import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.netty.KvStateRequestStats;
 import org.apache.flink.util.Preconditions;
 
-import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
-import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 
 /**
- * Netty-based server answering {@link KvStateRequest} messages.
- *
- * <p>Requests are handled by asynchronous query tasks (see {@link KvStateServerHandler.AsyncKvStateQueryTask})
- * that are executed by a separate query Thread pool. This pool is shared among
- * all TCP connections.
- *
- * <p>The incoming pipeline looks as follows:
- * <pre>
- * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
- * </pre>
- *
- * <p>Received binary messages are expected to contain a frame length field. Netty's
- * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
- * giving it to our {@link KvStateServerHandler}.
- *
- * <p>Connections are established and closed by the client. The server only
- * closes the connection on a fatal failure that cannot be recovered. A
- * server-side connection close is considered a failure by the client.
+ * The default implementation of the {@link KvStateServer}.
  */
-public class KvStateServerImpl implements KvStateServer {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateServer.class);
+@Internal
+public class KvStateServerImpl extends AbstractServerBase<KvStateInternalRequest, KvStateResponse> implements KvStateServer {
 
-	/** Server config: low water mark. */
-	private static final int LOW_WATER_MARK = 8 * 1024;
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateServerImpl.class);
 
-	/** Server config: high water mark. */
-	private static final int HIGH_WATER_MARK = 32 * 1024;
+	/** The {@link KvStateRegistry} to query for state instances. */
+	private final KvStateRegistry kvStateRegistry;
 
-	/** Netty's ServerBootstrap. */
-	private final ServerBootstrap bootstrap;
+	private final KvStateRequestStats stats;
 
-	/** Query executor thread pool. */
-	private final ExecutorService queryExecutor;
-
-	/** Address of this server. */
-	private KvStateServerAddress serverAddress;
+	private MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer;
 
 	/**
-	 * Creates the {@link KvStateServer}.
+	 * Creates the state server.
+	 *
+	 * <p>The server is instantiated using reflection by the
+	 * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateServer(InetAddress, int, int, int, KvStateRegistry, KvStateRequestStats)
+	 * QueryableStateUtils.startKvStateServer(InetAddress, int, int, int, KvStateRegistry, KvStateRequestStats)}.
 	 *
 	 * <p>The server needs to be started via {@link #start()} in order to bind
 	 * to the configured bind address.
 	 *
-	 * @param bindAddress         Address to bind to
-	 * @param bindPort            Port to bind to. Pick random port if 0.
-	 * @param numEventLoopThreads Number of event loop threads
-	 * @param numQueryThreads     Number of query threads
-	 * @param kvStateRegistry     KvStateRegistry to query for KvState instances
-	 * @param stats               Statistics tracker
+	 * @param bindAddress the address to listen to.
+	 * @param bindPort the port to listen to.
+	 * @param numEventLoopThreads number of event loop threads.
+	 * @param numQueryThreads number of query threads.
+	 * @param kvStateRegistry {@link KvStateRegistry} to query for state instances.
+	 * @param stats the statistics collector.
 	 */
 	public KvStateServerImpl(
-			InetAddress bindAddress,
-			Integer bindPort,
-			Integer numEventLoopThreads,
-			Integer numQueryThreads,
-			KvStateRegistry kvStateRegistry,
-			KvStateRequestStats stats) {
-
-		Preconditions.checkArgument(bindPort >= 0 && bindPort <= 65536, "Port " + bindPort +
-				" is out of valid port range (0-65536).");
-
-		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
-		Preconditions.checkArgument(numQueryThreads >= 1, "Non-positive number of query threads.");
-
-		Preconditions.checkNotNull(kvStateRegistry, "KvStateRegistry");
-		Preconditions.checkNotNull(stats, "KvStateRequestStats");
-
-		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
-
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateServer EventLoop Thread %d")
-				.build();
-
-		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
-
-		queryExecutor = createQueryExecutor(numQueryThreads);
-
-		// Shared between all channels
-		KvStateServerHandler serverHandler = new KvStateServerHandler(
-				kvStateRegistry,
-				queryExecutor,
-				stats);
-
-		bootstrap = new ServerBootstrap()
-				// Bind address and port
-				.localAddress(bindAddress, bindPort)
-				// NIO server channels
-				.group(nioGroup)
-				.channel(NioServerSocketChannel.class)
-				// Server channel Options
-				.option(ChannelOption.ALLOCATOR, bufferPool)
-				// Child channel options
-				.childOption(ChannelOption.ALLOCATOR, bufferPool)
-				.childOption(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, HIGH_WATER_MARK)
-				.childOption(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, LOW_WATER_MARK)
-				// See initializer for pipeline details
-				.childHandler(new KvStateServerChannelInitializer(serverHandler));
+			final InetAddress bindAddress,
+			final Integer bindPort,
+			final Integer numEventLoopThreads,
+			final Integer numQueryThreads,
+			final KvStateRegistry kvStateRegistry,
+			final KvStateRequestStats stats) {
+
+		super("Queryable State Server", bindAddress, bindPort, numEventLoopThreads, numQueryThreads);
+		this.stats = Preconditions.checkNotNull(stats);
+		this.kvStateRegistry = Preconditions.checkNotNull(kvStateRegistry);
 	}
 
 	@Override
-	public void start() throws InterruptedException {
-		Channel channel = bootstrap.bind().sync().channel();
-
-		InetSocketAddress localAddress = (InetSocketAddress) channel.localAddress();
-		serverAddress = new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
+	public AbstractServerHandler<KvStateInternalRequest, KvStateResponse> initializeHandler() {
+		this.serializer = new MessageSerializer<>(
+				new KvStateInternalRequest.KvStateInternalRequestDeserializer(),
+				new KvStateResponse.KvStateResponseDeserializer());
+		return new KvStateServerHandler(this, kvStateRegistry, serializer, stats);
 	}
 
-	@Override
-	public KvStateServerAddress getAddress() {
-		if (serverAddress == null) {
-			throw new IllegalStateException("KvStateServer not started yet.");
-		}
-
-		return serverAddress;
+	public MessageSerializer<KvStateInternalRequest, KvStateResponse> getSerializer() {
+		Preconditions.checkState(serializer != null, "Server " + getServerName() + " has not been started.");
+		return serializer;
 	}
 
 	@Override
-	public void shutDown() {
-		if (bootstrap != null) {
-			EventLoopGroup group = bootstrap.group();
-			if (group != null) {
-				group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
-			}
-		}
-
-		if (queryExecutor != null) {
-			queryExecutor.shutdown();
-		}
-
-		serverAddress = null;
+	public void start() throws InterruptedException {
+		super.start();
 	}
 
-	/**
-	 * Creates a thread pool for the query execution.
-	 *
-	 * @param numQueryThreads Number of query threads.
-	 * @return Thread pool for query execution
-	 */
-	private static ExecutorService createQueryExecutor(int numQueryThreads) {
-		ThreadFactory threadFactory = new ThreadFactoryBuilder()
-				.setDaemon(true)
-				.setNameFormat("Flink KvStateServer Query Thread %d")
-				.build();
-
-		return Executors.newFixedThreadPool(numQueryThreads, threadFactory);
+	@Override
+	public KvStateServerAddress getServerAddress() {
+		return super.getServerAddress();
 	}
 
-	/**
-	 * Channel pipeline initializer.
-	 *
-	 * <p>The request handler is shared, whereas the other handlers are created
-	 * per channel.
-	 */
-	private static final class KvStateServerChannelInitializer extends ChannelInitializer<SocketChannel> {
-
-		/** The shared request handler. */
-		private final KvStateServerHandler sharedRequestHandler;
-
-		/**
-		 * Creates the channel pipeline initializer with the shared request handler.
-		 *
-		 * @param sharedRequestHandler Shared request handler.
-		 */
-		public KvStateServerChannelInitializer(KvStateServerHandler sharedRequestHandler) {
-			this.sharedRequestHandler = Preconditions.checkNotNull(sharedRequestHandler, "Request handler");
-		}
-
-		@Override
-		protected void initChannel(SocketChannel ch) throws Exception {
-			ch.pipeline()
-					.addLast(new ChunkedWriteHandler())
-					.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-					.addLast(sharedRequestHandler);
-		}
+	@Override
+	public void shutdown() {
+		super.shutdown();
 	}
-
 }


[03/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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
deleted file mode 100644
index 4b73fbb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequest.java
+++ /dev/null
@@ -1,89 +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.query.netty.message;
-
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.state.internal.InternalKvState;
-import org.apache.flink.util.Preconditions;
-
-/**
- * A {@link InternalKvState} instance request for a specific key and namespace.
- */
-public final class KvStateRequest {
-
-	/** ID for this request. */
-	private final long requestId;
-
-	/** ID of the requested KvState instance. */
-	private final KvStateID kvStateId;
-
-	/** Serialized key and namespace to request from the KvState instance. */
-	private final byte[] serializedKeyAndNamespace;
-
-	/**
-	 * Creates a KvState instance request.
-	 *
-	 * @param requestId                 ID for this request
-	 * @param kvStateId                 ID of the requested KvState instance
-	 * @param serializedKeyAndNamespace Serialized key and namespace to request from the KvState
-	 *                                  instance
-	 */
-	KvStateRequest(long requestId, KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
-		this.requestId = requestId;
-		this.kvStateId = Preconditions.checkNotNull(kvStateId, "KvStateID");
-		this.serializedKeyAndNamespace = Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
-	}
-
-	/**
-	 * Returns the request ID.
-	 *
-	 * @return Request ID
-	 */
-	public long getRequestId() {
-		return requestId;
-	}
-
-	/**
-	 * Returns the ID of the requested KvState instance.
-	 *
-	 * @return ID of the requested KvState instance
-	 */
-	public KvStateID getKvStateId() {
-		return kvStateId;
-	}
-
-	/**
-	 * Returns the serialized key and namespace to request from the KvState
-	 * instance.
-	 *
-	 * @return Serialized key and namespace to request from the KvState instance
-	 */
-	public byte[] getSerializedKeyAndNamespace() {
-		return serializedKeyAndNamespace;
-	}
-
-	@Override
-	public String toString() {
-		return "KvStateRequest{" +
-				"requestId=" + requestId +
-				", kvStateId=" + kvStateId +
-				", serializedKeyAndNamespace.length=" + serializedKeyAndNamespace.length +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestFailure.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestFailure.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestFailure.java
deleted file mode 100644
index 06a3ce8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestFailure.java
+++ /dev/null
@@ -1,68 +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.query.netty.message;
-
-/**
- * A failure response to a {@link KvStateRequest}.
- */
-public final class KvStateRequestFailure {
-
-	/** ID of the request responding to. */
-	private final long requestId;
-
-	/** Failure cause. Not allowed to be a user type. */
-	private final Throwable cause;
-
-	/**
-	 * Creates a failure response to a {@link KvStateRequest}.
-	 *
-	 * @param requestId ID for the request responding to
-	 * @param cause     Failure cause (not allowed to be a user type)
-	 */
-	KvStateRequestFailure(long requestId, Throwable cause) {
-		this.requestId = requestId;
-		this.cause = cause;
-	}
-
-	/**
-	 * Returns the request ID responding to.
-	 *
-	 * @return Request ID responding to
-	 */
-	public long getRequestId() {
-		return requestId;
-	}
-
-	/**
-	 * Returns the failure cause.
-	 *
-	 * @return Failure cause
-	 */
-	public Throwable getCause() {
-		return cause;
-	}
-
-	@Override
-	public String toString() {
-		return "KvStateRequestFailure{" +
-				"requestId=" + requestId +
-				", cause=" + cause +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestResult.java
deleted file mode 100644
index 2bd8a36..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestResult.java
+++ /dev/null
@@ -1,74 +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.query.netty.message;
-
-import org.apache.flink.util.Preconditions;
-
-/**
- * A successful response to a {@link KvStateRequest} containing the serialized
- * result for the requested key and namespace.
- */
-public final class KvStateRequestResult {
-
-	/** ID of the request responding to. */
-	private final long requestId;
-
-	/**
-	 * Serialized result for the requested key and namespace. If no result was
-	 * available for the specified key and namespace, this is <code>null</code>.
-	 */
-	private final byte[] serializedResult;
-
-	/**
-	 * Creates a successful {@link KvStateRequestResult} response.
-	 *
-	 * @param requestId        ID of the request responding to
-	 * @param serializedResult Serialized result or <code>null</code> if none
-	 */
-	KvStateRequestResult(long requestId, byte[] serializedResult) {
-		this.requestId = requestId;
-		this.serializedResult = Preconditions.checkNotNull(serializedResult, "Serialization result");
-	}
-
-	/**
-	 * Returns the request ID responding to.
-	 *
-	 * @return Request ID responding to
-	 */
-	public long getRequestId() {
-		return requestId;
-	}
-
-	/**
-	 * Returns the serialized result or <code>null</code> if none available.
-	 *
-	 * @return Serialized result or <code>null</code> if none available.
-	 */
-	public byte[] getSerializedResult() {
-		return serializedResult;
-	}
-
-	@Override
-	public String toString() {
-		return "KvStateRequestResult{" +
-				"requestId=" + requestId +
-				", serializedResult.length=" + serializedResult.length +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
deleted file mode 100644
index 68f06e3..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestSerializer.java
+++ /dev/null
@@ -1,568 +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.query.netty.message;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.netty.KvStateClient;
-import org.apache.flink.runtime.query.netty.KvStateServer;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufInputStream;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufOutputStream;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Serialization and deserialization of messages exchanged between
- * {@link KvStateClient} and {@link KvStateServer}.
- *
- * <p>The binary messages have the following format:
- *
- * <pre>
- *                     <------ Frame ------------------------->
- *                    +----------------------------------------+
- *                    |        HEADER (8)      | PAYLOAD (VAR) |
- * +------------------+----------------------------------------+
- * | FRAME LENGTH (4) | VERSION (4) | TYPE (4) | CONTENT (VAR) |
- * +------------------+----------------------------------------+
- * </pre>
- *
- * <p>The concrete content of a message depends on the {@link KvStateRequestType}.
- */
-public final class KvStateRequestSerializer {
-
-	/** The serialization version ID. */
-	private static final int VERSION = 0x79a1b710;
-
-	/** Byte length of the header. */
-	private static final int HEADER_LENGTH = 8;
-
-	// ------------------------------------------------------------------------
-	// Serialization
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Allocates a buffer and serializes the KvState request into it.
-	 *
-	 * @param alloc                     ByteBuf allocator for the buffer to
-	 *                                  serialize message into
-	 * @param requestId                 ID for this request
-	 * @param kvStateId                 ID of the requested KvState instance
-	 * @param serializedKeyAndNamespace Serialized key and namespace to request
-	 *                                  from the KvState instance.
-	 * @return Serialized KvState request message
-	 */
-	public static ByteBuf serializeKvStateRequest(
-			ByteBufAllocator alloc,
-			long requestId,
-			KvStateID kvStateId,
-			byte[] serializedKeyAndNamespace) {
-
-		// Header + request ID + KvState ID + Serialized namespace
-		int frameLength = HEADER_LENGTH + 8 + (8 + 8) + (4 + serializedKeyAndNamespace.length);
-		ByteBuf buf = alloc.ioBuffer(frameLength + 4); // +4 for frame length
-
-		buf.writeInt(frameLength);
-
-		writeHeader(buf, KvStateRequestType.REQUEST);
-
-		buf.writeLong(requestId);
-		buf.writeLong(kvStateId.getLowerPart());
-		buf.writeLong(kvStateId.getUpperPart());
-		buf.writeInt(serializedKeyAndNamespace.length);
-		buf.writeBytes(serializedKeyAndNamespace);
-
-		return buf;
-	}
-
-	/**
-	 * Allocates a buffer and serializes the KvState request result into it.
-	 *
-	 * @param alloc             ByteBuf allocator for the buffer to serialize message into
-	 * @param requestId         ID for this request
-	 * @param serializedResult  Serialized Result
-	 * @return Serialized KvState request result message
-	 */
-	public static ByteBuf serializeKvStateRequestResult(
-			ByteBufAllocator alloc,
-			long requestId,
-			byte[] serializedResult) {
-
-		Preconditions.checkNotNull(serializedResult, "Serialized result");
-
-		// Header + request ID + serialized result
-		int frameLength = HEADER_LENGTH + 8 + 4 + serializedResult.length;
-
-		ByteBuf buf = alloc.ioBuffer(frameLength);
-
-		buf.writeInt(frameLength);
-		writeHeader(buf, KvStateRequestType.REQUEST_RESULT);
-		buf.writeLong(requestId);
-
-		buf.writeInt(serializedResult.length);
-		buf.writeBytes(serializedResult);
-
-		return buf;
-	}
-
-	/**
-	 * Allocates a buffer and serializes the KvState request failure into it.
-	 *
-	 * @param alloc ByteBuf allocator for the buffer to serialize message into
-	 * @param requestId ID of the request responding to
-	 * @param cause Failure cause
-	 * @return Serialized KvState request failure message
-	 * @throws IOException Serialization failures are forwarded
-	 */
-	public static ByteBuf serializeKvStateRequestFailure(
-			ByteBufAllocator alloc,
-			long requestId,
-			Throwable cause) throws IOException {
-
-		ByteBuf buf = alloc.ioBuffer();
-
-		// Frame length is set at the end
-		buf.writeInt(0);
-
-		writeHeader(buf, KvStateRequestType.REQUEST_FAILURE);
-
-		// Message
-		buf.writeLong(requestId);
-
-		try (ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
-				ObjectOutputStream out = new ObjectOutputStream(bbos)) {
-
-			out.writeObject(cause);
-		}
-
-		// Set frame length
-		int frameLength = buf.readableBytes() - 4;
-		buf.setInt(0, frameLength);
-
-		return buf;
-	}
-
-	/**
-	 * Allocates a buffer and serializes the server failure into it.
-	 *
-	 * <p>The cause must not be or contain any user types as causes.
-	 *
-	 * @param alloc ByteBuf allocator for the buffer to serialize message into
-	 * @param cause Failure cause
-	 * @return Serialized server failure message
-	 * @throws IOException Serialization failures are forwarded
-	 */
-	public static ByteBuf serializeServerFailure(ByteBufAllocator alloc, Throwable cause) throws IOException {
-		ByteBuf buf = alloc.ioBuffer();
-
-		// Frame length is set at end
-		buf.writeInt(0);
-
-		writeHeader(buf, KvStateRequestType.SERVER_FAILURE);
-
-		try (ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
-				ObjectOutputStream out = new ObjectOutputStream(bbos)) {
-
-			out.writeObject(cause);
-		}
-
-		// Set frame length
-		int frameLength = buf.readableBytes() - 4;
-		buf.setInt(0, frameLength);
-
-		return buf;
-	}
-
-	// ------------------------------------------------------------------------
-	// Deserialization
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Deserializes the header and returns the request type.
-	 *
-	 * @param buf Buffer to deserialize (expected to be at header position)
-	 * @return Deserialzied request type
-	 * @throws IllegalArgumentException If unexpected message version or message type
-	 */
-	public static KvStateRequestType deserializeHeader(ByteBuf buf) {
-		// Check the version
-		int version = buf.readInt();
-		if (version != VERSION) {
-			throw new IllegalArgumentException("Illegal message version " + version +
-					". Expected: " + VERSION + ".");
-		}
-
-		// Get the message type
-		int msgType = buf.readInt();
-		KvStateRequestType[] values = KvStateRequestType.values();
-		if (msgType >= 0 && msgType < values.length) {
-			return values[msgType];
-		} else {
-			throw new IllegalArgumentException("Illegal message type with index " + msgType);
-		}
-	}
-
-	/**
-	 * Deserializes the KvState request message.
-	 *
-	 * <p><strong>Important</strong>: the returned buffer is sliced from the
-	 * incoming ByteBuf stream and retained. Therefore, it needs to be recycled
-	 * by the consumer.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequest
-	 */
-	public static KvStateRequest deserializeKvStateRequest(ByteBuf buf) {
-		long requestId = buf.readLong();
-		KvStateID kvStateId = new KvStateID(buf.readLong(), buf.readLong());
-
-		// Serialized key and namespace
-		int length = buf.readInt();
-
-		if (length < 0) {
-			throw new IllegalArgumentException("Negative length for serialized key and namespace. " +
-					"This indicates a serialization error.");
-		}
-
-		// Copy the buffer in order to be able to safely recycle the ByteBuf
-		byte[] serializedKeyAndNamespace = new byte[length];
-		if (length > 0) {
-			buf.readBytes(serializedKeyAndNamespace);
-		}
-
-		return new KvStateRequest(requestId, kvStateId, serializedKeyAndNamespace);
-	}
-
-	/**
-	 * Deserializes the KvState request result.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequestResult
-	 */
-	public static KvStateRequestResult deserializeKvStateRequestResult(ByteBuf buf) {
-		long requestId = buf.readLong();
-
-		// Serialized KvState
-		int length = buf.readInt();
-
-		if (length < 0) {
-			throw new IllegalArgumentException("Negative length for serialized result. " +
-					"This indicates a serialization error.");
-		}
-
-		byte[] serializedValue = new byte[length];
-
-		if (length > 0) {
-			buf.readBytes(serializedValue);
-		}
-
-		return new KvStateRequestResult(requestId, serializedValue);
-	}
-
-	/**
-	 * Deserializes the KvState request failure.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequestFailure
-	 */
-	public static KvStateRequestFailure deserializeKvStateRequestFailure(ByteBuf buf) throws IOException, ClassNotFoundException {
-		long requestId = buf.readLong();
-
-		Throwable cause;
-		try (ByteBufInputStream bbis = new ByteBufInputStream(buf);
-				ObjectInputStream in = new ObjectInputStream(bbis)) {
-
-			cause = (Throwable) in.readObject();
-		}
-
-		return new KvStateRequestFailure(requestId, cause);
-	}
-
-	/**
-	 * Deserializes the KvState request failure.
-	 *
-	 * @param buf Buffer to deserialize (expected to be positioned after header)
-	 * @return Deserialized KvStateRequestFailure
-	 * @throws IOException            Serialization failure are forwarded
-	 * @throws ClassNotFoundException If Exception type can not be loaded
-	 */
-	public static Throwable deserializeServerFailure(ByteBuf buf) throws IOException, ClassNotFoundException {
-		try (ByteBufInputStream bbis = new ByteBufInputStream(buf);
-				ObjectInputStream in = new ObjectInputStream(bbis)) {
-
-			return (Throwable) in.readObject();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// Generic serialization utils
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Serializes the key and namespace into a {@link ByteBuffer}.
-	 *
-	 * <p>The serialized format matches the RocksDB state backend key format, i.e.
-	 * the key and namespace don't have to be deserialized for RocksDB lookups.
-	 *
-	 * @param key                 Key to serialize
-	 * @param keySerializer       Serializer for the key
-	 * @param namespace           Namespace to serialize
-	 * @param namespaceSerializer Serializer for the namespace
-	 * @param <K>                 Key type
-	 * @param <N>                 Namespace type
-	 * @return Buffer holding the serialized key and namespace
-	 * @throws IOException Serialization errors are forwarded
-	 */
-	public static <K, N> byte[] serializeKeyAndNamespace(
-			K key,
-			TypeSerializer<K> keySerializer,
-			N namespace,
-			TypeSerializer<N> namespaceSerializer) throws IOException {
-
-		DataOutputSerializer dos = new DataOutputSerializer(32);
-
-		keySerializer.serialize(key, dos);
-		dos.writeByte(42);
-		namespaceSerializer.serialize(namespace, dos);
-
-		return dos.getCopyOfBuffer();
-	}
-
-	/**
-	 * Deserializes the key and namespace into a {@link Tuple2}.
-	 *
-	 * @param serializedKeyAndNamespace Serialized key and namespace
-	 * @param keySerializer             Serializer for the key
-	 * @param namespaceSerializer       Serializer for the namespace
-	 * @param <K>                       Key type
-	 * @param <N>                       Namespace
-	 * @return Tuple2 holding deserialized key and namespace
-	 * @throws IOException              if the deserialization fails for any reason
-	 */
-	public static <K, N> Tuple2<K, N> deserializeKeyAndNamespace(
-			byte[] serializedKeyAndNamespace,
-			TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer) throws IOException {
-
-		DataInputDeserializer dis = new DataInputDeserializer(
-				serializedKeyAndNamespace,
-				0,
-				serializedKeyAndNamespace.length);
-
-		try {
-			K key = keySerializer.deserialize(dis);
-			byte magicNumber = dis.readByte();
-			if (magicNumber != 42) {
-				throw new IOException("Unexpected magic number " + magicNumber + ".");
-			}
-			N namespace = namespaceSerializer.deserialize(dis);
-
-			if (dis.available() > 0) {
-				throw new IOException("Unconsumed bytes in the serialized key and namespace.");
-			}
-
-			return new Tuple2<>(key, namespace);
-		} catch (IOException e) {
-			throw new IOException("Unable to deserialize key " +
-				"and namespace. This indicates a mismatch in the key/namespace " +
-				"serializers used by the KvState instance and this access.", e);
-		}
-	}
-
-	/**
-	 * Serializes the value with the given serializer.
-	 *
-	 * @param value      Value of type T to serialize
-	 * @param serializer Serializer for T
-	 * @param <T>        Type of the value
-	 * @return Serialized value or <code>null</code> if value <code>null</code>
-	 * @throws IOException On failure during serialization
-	 */
-	public static <T> byte[] serializeValue(T value, TypeSerializer<T> serializer) throws IOException {
-		if (value != null) {
-			// Serialize
-			DataOutputSerializer dos = new DataOutputSerializer(32);
-			serializer.serialize(value, dos);
-			return dos.getCopyOfBuffer();
-		} else {
-			return null;
-		}
-	}
-
-	/**
-	 * Deserializes the value with the given serializer.
-	 *
-	 * @param serializedValue Serialized value of type T
-	 * @param serializer      Serializer for T
-	 * @param <T>             Type of the value
-	 * @return Deserialized value or <code>null</code> if the serialized value
-	 * is <code>null</code>
-	 * @throws IOException On failure during deserialization
-	 */
-	public static <T> T deserializeValue(byte[] serializedValue, TypeSerializer<T> serializer) throws IOException {
-		if (serializedValue == null) {
-			return null;
-		} else {
-			final DataInputDeserializer deser = new DataInputDeserializer(
-				serializedValue, 0, serializedValue.length);
-			final T value = serializer.deserialize(deser);
-			if (deser.available() > 0) {
-				throw new IOException(
-					"Unconsumed bytes in the deserialized value. " +
-						"This indicates a mismatch in the value serializers " +
-						"used by the KvState instance and this access.");
-			}
-			return value;
-		}
-	}
-
-	/**
-	 * Deserializes all values with the given serializer.
-	 *
-	 * @param serializedValue Serialized value of type List&lt;T&gt;
-	 * @param serializer      Serializer for T
-	 * @param <T>             Type of the value
-	 * @return Deserialized list or <code>null</code> if the serialized value
-	 * is <code>null</code>
-	 * @throws IOException On failure during deserialization
-	 */
-	public static <T> List<T> deserializeList(byte[] serializedValue, TypeSerializer<T> serializer) throws IOException {
-		if (serializedValue != null) {
-			final DataInputDeserializer in = new DataInputDeserializer(
-				serializedValue, 0, serializedValue.length);
-
-			try {
-				final List<T> result = new ArrayList<>();
-				while (in.available() > 0) {
-					result.add(serializer.deserialize(in));
-
-					// The expected binary format has a single byte separator. We
-					// want a consistent binary format in order to not need any
-					// special casing during deserialization. A "cleaner" format
-					// would skip this extra byte, but would require a memory copy
-					// for RocksDB, which stores the data serialized in this way
-					// for lists.
-					if (in.available() > 0) {
-						in.readByte();
-					}
-				}
-
-				return result;
-			} catch (IOException e) {
-				throw new IOException(
-						"Unable to deserialize value. " +
-							"This indicates a mismatch in the value serializers " +
-							"used by the KvState instance and this access.", e);
-			}
-		} else {
-			return null;
-		}
-	}
-
-	/**
-	 * Serializes all values of the Iterable with the given serializer.
-	 *
-	 * @param entries         Key-value pairs to serialize
-	 * @param keySerializer   Serializer for UK
-	 * @param valueSerializer Serializer for UV
-	 * @param <UK>            Type of the keys
-	 * @param <UV>            Type of the values
-	 * @return Serialized values or <code>null</code> if values <code>null</code> or empty
-	 * @throws IOException On failure during serialization
-	 */
-	public static <UK, UV> byte[] serializeMap(Iterable<Map.Entry<UK, UV>> entries, TypeSerializer<UK> keySerializer, TypeSerializer<UV> valueSerializer) throws IOException {
-		if (entries != null) {
-			// Serialize
-			DataOutputSerializer dos = new DataOutputSerializer(32);
-
-			for (Map.Entry<UK, UV> entry : entries) {
-				keySerializer.serialize(entry.getKey(), dos);
-
-				if (entry.getValue() == null) {
-					dos.writeBoolean(true);
-				} else {
-					dos.writeBoolean(false);
-					valueSerializer.serialize(entry.getValue(), dos);
-				}
-			}
-
-			return dos.getCopyOfBuffer();
-		} else {
-			return null;
-		}
-	}
-
-	/**
-	 * Deserializes all kv pairs with the given serializer.
-	 *
-	 * @param serializedValue Serialized value of type Map&lt;UK, UV&gt;
-	 * @param keySerializer   Serializer for UK
-	 * @param valueSerializer Serializer for UV
-	 * @param <UK>            Type of the key
-	 * @param <UV>            Type of the value.
-	 * @return Deserialized map or <code>null</code> if the serialized value
-	 * is <code>null</code>
-	 * @throws IOException On failure during deserialization
-	 */
-	public static <UK, UV> Map<UK, UV> deserializeMap(byte[] serializedValue, TypeSerializer<UK> keySerializer, TypeSerializer<UV> valueSerializer) throws IOException {
-		if (serializedValue != null) {
-			DataInputDeserializer in = new DataInputDeserializer(serializedValue, 0, serializedValue.length);
-
-			Map<UK, UV> result = new HashMap<>();
-			while (in.available() > 0) {
-				UK key = keySerializer.deserialize(in);
-
-				boolean isNull = in.readBoolean();
-				UV value = isNull ? null : valueSerializer.deserialize(in);
-
-				result.put(key, value);
-			}
-
-			return result;
-		} else {
-			return null;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Helper for writing the header.
-	 *
-	 * @param buf         Buffer to serialize header into
-	 * @param requestType Result type to serialize
-	 */
-	private static void writeHeader(ByteBuf buf, KvStateRequestType requestType) {
-		buf.writeInt(VERSION);
-		buf.writeInt(requestType.ordinal());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestType.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestType.java
deleted file mode 100644
index de7270a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateRequestType.java
+++ /dev/null
@@ -1,40 +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.query.netty.message;
-
-import org.apache.flink.runtime.query.netty.KvStateServer;
-
-/**
- * Expected message types when communicating with the {@link KvStateServer}.
- */
-public enum KvStateRequestType {
-
-	/** Request a KvState instance. */
-	REQUEST,
-
-	/** Successful response to a KvStateRequest. */
-	REQUEST_RESULT,
-
-	/** Failure response to a KvStateRequest. */
-	REQUEST_FAILURE,
-
-	/** Generic server failure. */
-	SERVER_FAILURE
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java
new file mode 100644
index 0000000..44ee571
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/message/KvStateSerializer.java
@@ -0,0 +1,267 @@
+/*
+ * 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.query.netty.message;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.runtime.util.DataOutputSerializer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization the different state types and namespaces.
+ */
+public final class KvStateSerializer {
+
+	// ------------------------------------------------------------------------
+	// Generic serialization utils
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Serializes the key and namespace into a {@link ByteBuffer}.
+	 *
+	 * <p>The serialized format matches the RocksDB state backend key format, i.e.
+	 * the key and namespace don't have to be deserialized for RocksDB lookups.
+	 *
+	 * @param key                 Key to serialize
+	 * @param keySerializer       Serializer for the key
+	 * @param namespace           Namespace to serialize
+	 * @param namespaceSerializer Serializer for the namespace
+	 * @param <K>                 Key type
+	 * @param <N>                 Namespace type
+	 * @return Buffer holding the serialized key and namespace
+	 * @throws IOException Serialization errors are forwarded
+	 */
+	public static <K, N> byte[] serializeKeyAndNamespace(
+			K key,
+			TypeSerializer<K> keySerializer,
+			N namespace,
+			TypeSerializer<N> namespaceSerializer) throws IOException {
+
+		DataOutputSerializer dos = new DataOutputSerializer(32);
+
+		keySerializer.serialize(key, dos);
+		dos.writeByte(42);
+		namespaceSerializer.serialize(namespace, dos);
+
+		return dos.getCopyOfBuffer();
+	}
+
+	/**
+	 * Deserializes the key and namespace into a {@link Tuple2}.
+	 *
+	 * @param serializedKeyAndNamespace Serialized key and namespace
+	 * @param keySerializer             Serializer for the key
+	 * @param namespaceSerializer       Serializer for the namespace
+	 * @param <K>                       Key type
+	 * @param <N>                       Namespace
+	 * @return Tuple2 holding deserialized key and namespace
+	 * @throws IOException              if the deserialization fails for any reason
+	 */
+	public static <K, N> Tuple2<K, N> deserializeKeyAndNamespace(
+			byte[] serializedKeyAndNamespace,
+			TypeSerializer<K> keySerializer,
+			TypeSerializer<N> namespaceSerializer) throws IOException {
+
+		DataInputDeserializer dis = new DataInputDeserializer(
+				serializedKeyAndNamespace,
+				0,
+				serializedKeyAndNamespace.length);
+
+		try {
+			K key = keySerializer.deserialize(dis);
+			byte magicNumber = dis.readByte();
+			if (magicNumber != 42) {
+				throw new IOException("Unexpected magic number " + magicNumber + ".");
+			}
+			N namespace = namespaceSerializer.deserialize(dis);
+
+			if (dis.available() > 0) {
+				throw new IOException("Unconsumed bytes in the serialized key and namespace.");
+			}
+
+			return new Tuple2<>(key, namespace);
+		} catch (IOException e) {
+			throw new IOException("Unable to deserialize key " +
+				"and namespace. This indicates a mismatch in the key/namespace " +
+				"serializers used by the KvState instance and this access.", e);
+		}
+	}
+
+	/**
+	 * Serializes the value with the given serializer.
+	 *
+	 * @param value      Value of type T to serialize
+	 * @param serializer Serializer for T
+	 * @param <T>        Type of the value
+	 * @return Serialized value or <code>null</code> if value <code>null</code>
+	 * @throws IOException On failure during serialization
+	 */
+	public static <T> byte[] serializeValue(T value, TypeSerializer<T> serializer) throws IOException {
+		if (value != null) {
+			// Serialize
+			DataOutputSerializer dos = new DataOutputSerializer(32);
+			serializer.serialize(value, dos);
+			return dos.getCopyOfBuffer();
+		} else {
+			return null;
+		}
+	}
+
+	/**
+	 * Deserializes the value with the given serializer.
+	 *
+	 * @param serializedValue Serialized value of type T
+	 * @param serializer      Serializer for T
+	 * @param <T>             Type of the value
+	 * @return Deserialized value or <code>null</code> if the serialized value
+	 * is <code>null</code>
+	 * @throws IOException On failure during deserialization
+	 */
+	public static <T> T deserializeValue(byte[] serializedValue, TypeSerializer<T> serializer) throws IOException {
+		if (serializedValue == null) {
+			return null;
+		} else {
+			final DataInputDeserializer deser = new DataInputDeserializer(
+				serializedValue, 0, serializedValue.length);
+			final T value = serializer.deserialize(deser);
+			if (deser.available() > 0) {
+				throw new IOException(
+					"Unconsumed bytes in the deserialized value. " +
+						"This indicates a mismatch in the value serializers " +
+						"used by the KvState instance and this access.");
+			}
+			return value;
+		}
+	}
+
+	/**
+	 * Deserializes all values with the given serializer.
+	 *
+	 * @param serializedValue Serialized value of type List&lt;T&gt;
+	 * @param serializer      Serializer for T
+	 * @param <T>             Type of the value
+	 * @return Deserialized list or <code>null</code> if the serialized value
+	 * is <code>null</code>
+	 * @throws IOException On failure during deserialization
+	 */
+	public static <T> List<T> deserializeList(byte[] serializedValue, TypeSerializer<T> serializer) throws IOException {
+		if (serializedValue != null) {
+			final DataInputDeserializer in = new DataInputDeserializer(
+				serializedValue, 0, serializedValue.length);
+
+			try {
+				final List<T> result = new ArrayList<>();
+				while (in.available() > 0) {
+					result.add(serializer.deserialize(in));
+
+					// The expected binary format has a single byte separator. We
+					// want a consistent binary format in order to not need any
+					// special casing during deserialization. A "cleaner" format
+					// would skip this extra byte, but would require a memory copy
+					// for RocksDB, which stores the data serialized in this way
+					// for lists.
+					if (in.available() > 0) {
+						in.readByte();
+					}
+				}
+
+				return result;
+			} catch (IOException e) {
+				throw new IOException(
+						"Unable to deserialize value. " +
+							"This indicates a mismatch in the value serializers " +
+							"used by the KvState instance and this access.", e);
+			}
+		} else {
+			return null;
+		}
+	}
+
+	/**
+	 * Serializes all values of the Iterable with the given serializer.
+	 *
+	 * @param entries         Key-value pairs to serialize
+	 * @param keySerializer   Serializer for UK
+	 * @param valueSerializer Serializer for UV
+	 * @param <UK>            Type of the keys
+	 * @param <UV>            Type of the values
+	 * @return Serialized values or <code>null</code> if values <code>null</code> or empty
+	 * @throws IOException On failure during serialization
+	 */
+	public static <UK, UV> byte[] serializeMap(Iterable<Map.Entry<UK, UV>> entries, TypeSerializer<UK> keySerializer, TypeSerializer<UV> valueSerializer) throws IOException {
+		if (entries != null) {
+			// Serialize
+			DataOutputSerializer dos = new DataOutputSerializer(32);
+
+			for (Map.Entry<UK, UV> entry : entries) {
+				keySerializer.serialize(entry.getKey(), dos);
+
+				if (entry.getValue() == null) {
+					dos.writeBoolean(true);
+				} else {
+					dos.writeBoolean(false);
+					valueSerializer.serialize(entry.getValue(), dos);
+				}
+			}
+
+			return dos.getCopyOfBuffer();
+		} else {
+			return null;
+		}
+	}
+
+	/**
+	 * Deserializes all kv pairs with the given serializer.
+	 *
+	 * @param serializedValue Serialized value of type Map&lt;UK, UV&gt;
+	 * @param keySerializer   Serializer for UK
+	 * @param valueSerializer Serializer for UV
+	 * @param <UK>            Type of the key
+	 * @param <UV>            Type of the value.
+	 * @return Deserialized map or <code>null</code> if the serialized value
+	 * is <code>null</code>
+	 * @throws IOException On failure during deserialization
+	 */
+	public static <UK, UV> Map<UK, UV> deserializeMap(byte[] serializedValue, TypeSerializer<UK> keySerializer, TypeSerializer<UV> valueSerializer) throws IOException {
+		if (serializedValue != null) {
+			DataInputDeserializer in = new DataInputDeserializer(serializedValue, 0, serializedValue.length);
+
+			Map<UK, UV> result = new HashMap<>();
+			while (in.available() > 0) {
+				UK key = keySerializer.deserialize(in);
+
+				boolean isNull = in.readBoolean();
+				UV value = isNull ? null : valueSerializer.deserialize(in);
+
+				result.put(key, value);
+			}
+
+			return result;
+		} else {
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/package-info.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/package-info.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/package-info.java
deleted file mode 100644
index 7e8de40..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/package-info.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.
- */
-
-/**
- * This package contains all Netty-based client/server classes used to query
- * KvState instances.
- *
- * <h2>Server and Client</h2>
- *
- * <p>Both server and client expect received binary messages to contain a frame
- * length field. Netty's {@link io.netty.handler.codec.LengthFieldBasedFrameDecoder}
- * is used to fully receive the frame before giving it to the respective client
- * or server handler.
- *
- * <p>Connection establishment and release happens by the client. The server
- * only closes a connection if a fatal failure happens that cannot be resolved
- * otherwise.
- *
- * <p>The is a single server per task manager and a single client can be shared
- * by multiple Threads.
- *
- * <p>See also:
- * <ul>
- * <li>{@link org.apache.flink.runtime.query.netty.KvStateServer}</li>
- * <li>{@link org.apache.flink.runtime.query.netty.KvStateServerHandler}</li>
- * <li>{@link org.apache.flink.runtime.query.netty.KvStateClient}</li>
- * <li>{@link org.apache.flink.runtime.query.netty.KvStateClientHandler}</li>
- * </ul>
- *
- * <h2>Serialization</h2>
- *
- * <p>The exchanged binary messages have the following format:
- *
- * <pre>
- *                     <------ Frame ------------------------->
- *                    +----------------------------------------+
- *                    |        HEADER (8)      | PAYLOAD (VAR) |
- * +------------------+----------------------------------------+
- * | FRAME LENGTH (4) | VERSION (4) | TYPE (4) | CONTENT (VAR) |
- * +------------------+----------------------------------------+
- * </pre>
- *
- * <p>For frame decoding, both server and client use Netty's {@link
- * io.netty.handler.codec.LengthFieldBasedFrameDecoder}. Message serialization
- * is done via static helpers in {@link org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer}.
- * The serialization helpers return {@link io.netty.buffer.ByteBuf} instances,
- * which are ready to be sent to the client or server respectively as they
- * contain the frame length.
- *
- * <p>See also:
- * <ul>
- * <li>{@link org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer}</li>
- * </ul>
- *
- * <h2>Statistics</h2>
- *
- * <p>Both server and client keep track of request statistics via {@link
- * org.apache.flink.runtime.query.netty.KvStateRequestStats}.
- *
- * <p>See also:
- * <ul>
- * <li>{@link org.apache.flink.runtime.query.netty.KvStateRequestStats}</li>
- * </ul>
- */
-package org.apache.flink.runtime.query.netty;

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/query/package-info.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/package-info.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/package-info.java
deleted file mode 100644
index 07a4396..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/package-info.java
+++ /dev/null
@@ -1,60 +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.
- */
-
-/**
- * This package contains all KvState query related classes.
- *
- * <h2>TaskManager and JobManager</h2>
- *
- * <p>State backends register queryable state instances at the {@link
- * org.apache.flink.runtime.query.KvStateRegistry}.
- * There is one registry per TaskManager. Registered KvState instances are
- * reported to the JobManager, where they are aggregated at the {@link
- * org.apache.flink.runtime.query.KvStateLocationRegistry}.
- *
- * <p>Instances of {@link org.apache.flink.runtime.query.KvStateLocation} contain
- * all information needed for a client to query a KvState instance.
- *
- * <p>See also:
- * <ul>
- * <li>{@link org.apache.flink.runtime.query.KvStateRegistry}</li>
- * <li>{@link org.apache.flink.runtime.query.TaskKvStateRegistry}</li>
- * <li>{@link org.apache.flink.runtime.query.KvStateLocation}</li>
- * <li>{@link org.apache.flink.runtime.query.KvStateLocationRegistry}</li>
- * </ul>
- *
- * <h2>Client</h2>
- *
- * The {@link org.apache.flink.runtime.query.QueryableStateClient} is used
- * to query KvState instances. The client takes care of {@link
- * org.apache.flink.runtime.query.KvStateLocation} lookup and caching. Queries
- * are then dispatched via the network client.
- *
- * <h3>JobManager Communication</h3>
- *
- * <p>The JobManager is queried for {@link org.apache.flink.runtime.query.KvStateLocation}
- * instances via the {@link org.apache.flink.runtime.query.KvStateLocationLookupService}.
- * The client caches resolved locations and dispatches queries directly to the
- * TaskManager.
- *
- * <h3>TaskManager Communication</h3>
- *
- * <p>After the location has been resolved, the TaskManager is queried via the
- * {@link org.apache.flink.runtime.query.netty.KvStateClient}.
- */
-package org.apache.flink.runtime.query;

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 7e1123d..97b6bcd 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,7 +24,7 @@ import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
 
@@ -90,7 +90,7 @@ public abstract class AbstractHeapState<K, N, SV, S extends State, SD extends St
 	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
 		Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
 
-		Tuple2<K, N> keyAndNamespace = KvStateRequestSerializer.deserializeKeyAndNamespace(
+		Tuple2<K, N> keyAndNamespace = KvStateSerializer.deserializeKeyAndNamespace(
 				serializedKeyAndNamespace, keySerializer, namespaceSerializer);
 
 		return getSerializedValue(keyAndNamespace.f0, keyAndNamespace.f1);
@@ -108,7 +108,7 @@ public abstract class AbstractHeapState<K, N, SV, S extends State, SD extends St
 
 		@SuppressWarnings("unchecked,rawtypes")
 		TypeSerializer serializer = stateDesc.getSerializer();
-		return KvStateRequestSerializer.serializeValue(result, serializer);
+		return KvStateSerializer.serializeValue(result, serializer);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java
index f393237..f981b9f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.state.heap;
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.util.Preconditions;
 
@@ -153,6 +153,6 @@ public class HeapMapState<K, N, UK, UV>
 		TypeSerializer<UK> userKeySerializer = stateDesc.getKeySerializer();
 		TypeSerializer<UV> userValueSerializer = stateDesc.getValueSerializer();
 
-		return KvStateRequestSerializer.serializeMap(result.entrySet(), userKeySerializer, userValueSerializer);
+		return KvStateSerializer.serializeMap(result.entrySet(), userKeySerializer, userValueSerializer);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
index fa1ae54..37d28de 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/QueryableStateConfiguration.java
@@ -47,7 +47,7 @@ public class QueryableStateConfiguration {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Returns whether queryable state is enabled. 
+	 * Returns whether queryable state is enabled.
 	 */
 	public boolean enabled() {
 		return enabled;
@@ -70,7 +70,7 @@ public class QueryableStateConfiguration {
 
 	/**
 	 * Returns the number of threads for the thread pool that performs the actual state lookup.
-	 * These threads perform the actual state lookup. 
+	 * These threads perform the actual state lookup.
 	 */
 	public int numQueryThreads() {
 		return numQueryThreads;
@@ -90,7 +90,7 @@ public class QueryableStateConfiguration {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Gets the configuration describing the queryable state as deactivated. 
+	 * Gets the configuration describing the queryable state as deactivated.
 	 */
 	public static QueryableStateConfiguration disabled() {
 		return new QueryableStateConfiguration(false, 0, 0, 0);

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
index 1c30ff6..7c5c830 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
@@ -39,8 +39,9 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
 import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.QueryableStateUtils;
 import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServer;
 import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable;
 import org.apache.flink.runtime.taskexecutor.slot.TimerService;
 import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer;
@@ -48,6 +49,7 @@ import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -138,7 +140,7 @@ public class TaskManagerServices {
 	public FileCache getFileCache() {
 		return fileCache;
 	}
-	
+
 	public TaskSlotTable getTaskSlotTable() {
 		return taskSlotTable;
 	}
@@ -214,7 +216,7 @@ public class TaskManagerServices {
 		final JobManagerTable jobManagerTable = new JobManagerTable();
 
 		final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation);
-		
+
 		return new TaskManagerServices(
 			taskManagerLocation,
 			memoryManager,
@@ -354,7 +356,7 @@ public class TaskManagerServices {
 		TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher();
 
 		KvStateRegistry kvStateRegistry = new KvStateRegistry();
-		KvStateServer kvStateServer;
+		KvStateServer kvStateServer = null;
 
 		if (taskManagerServicesConfiguration.getQueryableStateConfig().enabled()) {
 			QueryableStateConfiguration qsConfig = taskManagerServicesConfiguration.getQueryableStateConfig();
@@ -365,15 +367,13 @@ public class TaskManagerServices {
 			int numQueryThreads = qsConfig.numQueryThreads() == 0 ?
 					taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numQueryThreads();
 
-			kvStateServer = new KvStateServer(
-				taskManagerServicesConfiguration.getTaskManagerAddress(),
-				qsConfig.port(),
-				numNetworkThreads,
-				numQueryThreads,
-				kvStateRegistry,
-				new DisabledKvStateRequestStats());
-		} else {
-			kvStateServer = null;
+			kvStateServer = QueryableStateUtils.createKvStateServer(
+					taskManagerServicesConfiguration.getTaskManagerAddress(),
+					qsConfig.port(),
+					numNetworkThreads,
+					numQueryThreads,
+					kvStateRegistry,
+					new DisabledKvStateRequestStats());
 		}
 
 		// we start the network first, to make sure it can allocate its buffers first
@@ -395,7 +395,7 @@ public class TaskManagerServices {
 	 * Calculates the amount of memory used for network buffers based on the total memory to use and
 	 * the according configuration parameters.
 	 *
-	 * The following configuration parameters are involved:
+	 * <p>The following configuration parameters are involved:
 	 * <ul>
 	 *  <li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_FRACTION},</li>
 	 * 	<li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_MIN},</li>
@@ -458,11 +458,11 @@ public class TaskManagerServices {
 	 * Calculates the amount of memory used for network buffers inside the current JVM instance
 	 * based on the available heap or the max heap size and the according configuration parameters.
 	 *
-	 * For containers or when started via scripts, if started with a memory limit and set to use
+	 * <p>For containers or when started via scripts, if started with a memory limit and set to use
 	 * off-heap memory, the maximum heap size for the JVM is adjusted accordingly and we are able
 	 * to extract the intended values from this.
 	 *
-	 * The following configuration parameters are involved:
+	 * <p>The following configuration parameters are involved:
 	 * <ul>
 	 *  <li>{@link TaskManagerOptions#MANAGED_MEMORY_FRACTION},</li>
 	 *  <li>{@link TaskManagerOptions#NETWORK_BUFFERS_MEMORY_FRACTION},</li>
@@ -629,7 +629,7 @@ public class TaskManagerServices {
 				if (LOG.isInfoEnabled()) {
 					long totalSpaceGb = file.getTotalSpace() >> 30;
 					long usableSpaceGb = file.getUsableSpace() >> 30;
-					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					double usablePercentage = (double) usableSpaceGb / totalSpaceGb * 100;
 					String path = file.getAbsolutePath();
 					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
 						path, totalSpaceGb, usableSpaceGb, usablePercentage));

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
index 6cc7569..f1f7d39 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java
@@ -31,6 +31,7 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
 import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
 import org.apache.flink.util.MathUtils;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,7 +44,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * Configuration for the task manager services such as the network environment, the memory manager,
- * the io manager and the metric registry
+ * the io manager and the metric registry.
  */
 public class TaskManagerServicesConfiguration {
 	private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServicesConfiguration.class);
@@ -106,7 +107,6 @@ public class TaskManagerServicesConfiguration {
 	//  Getter/Setter
 	// --------------------------------------------------------------------------------------------
 
-
 	public InetAddress getTaskManagerAddress() {
 		return taskManagerAddress;
 	}
@@ -291,7 +291,7 @@ public class TaskManagerServicesConfiguration {
 
 		if (!hasNewNetworkBufConf(configuration)) {
 			// map old config to new one:
-			networkBufMin = networkBufMax = ((long)numNetworkBuffers) * pageSize;
+			networkBufMin = networkBufMax = ((long) numNetworkBuffers) * pageSize;
 		} else {
 			if (configuration.contains(TaskManagerOptions.NETWORK_NUM_BUFFERS)) {
 				LOG.info("Ignoring old (but still present) network buffer configuration via {}.",
@@ -439,9 +439,8 @@ public class TaskManagerServicesConfiguration {
 	static void checkConfigParameter(boolean condition, Object parameter, String name, String errorMessage)
 			throws IllegalConfigurationException {
 		if (!condition) {
-			throw new IllegalConfigurationException("Invalid configuration value for " + 
+			throw new IllegalConfigurationException("Invalid configuration value for " +
 					name + " : " + parameter + " - " + errorMessage);
 		}
 	}
 }
-

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/test/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupServiceTest.java
deleted file mode 100644
index edefcf8..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/AkkaKvStateLocationLookupServiceTest.java
+++ /dev/null
@@ -1,398 +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.query;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.akka.FlinkUntypedActor;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
-import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.LookupRetryStrategy;
-import org.apache.flink.runtime.query.AkkaKvStateLocationLookupService.LookupRetryStrategyFactory;
-import org.apache.flink.runtime.query.KvStateMessage.LookupKvStateLocation;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.TestLogger;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.Props;
-import akka.actor.Status;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link AkkaKvStateLocationLookupService}.
- */
-public class AkkaKvStateLocationLookupServiceTest extends TestLogger {
-
-	/** The default timeout. */
-	private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
-
-	/** Test actor system shared between the tests. */
-	private static ActorSystem testActorSystem;
-
-	@BeforeClass
-	public static void setUp() throws Exception {
-		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-	}
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
-	}
-
-	/**
-	 * Tests responses if no leader notification has been reported or leadership
-	 * has been lost (leaderAddress = <code>null</code>).
-	 */
-	@Test
-	public void testNoJobManagerRegistered() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			null,
-			null);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		//
-		// No leader registered initially => fail with UnknownJobManager
-		//
-		try {
-			JobID jobId = new JobID();
-			String name = "coffee";
-
-			Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(jobId, name);
-
-			Await.result(locationFuture, TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (UnknownJobManager ignored) {
-			// Expected
-		}
-
-		assertEquals("Received unexpected lookup", 0, received.size());
-
-		//
-		// Leader registration => communicate with new leader
-		//
-		UUID leaderSessionId = HighAvailabilityServices.DEFAULT_LEADER_ID;
-		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "tea");
-
-		ActorRef testActor = LookupResponseActor.create(received, leaderSessionId, expected);
-
-		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		// Notify the service about a leader
-		leaderRetrievalService.notifyListener(testActorAddress, leaderSessionId);
-
-		JobID jobId = new JobID();
-		String name = "tea";
-
-		// Verify that the leader response is handled
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, name), TIMEOUT);
-		assertEquals(expected, location);
-
-		// Verify that the correct message was sent to the leader
-		assertEquals(1, received.size());
-
-		verifyLookupMsg(received.poll(), jobId, name);
-
-		//
-		// Leader loss => fail with UnknownJobManager
-		//
-		leaderRetrievalService.notifyListener(null, null);
-
-		try {
-			Future<KvStateLocation> locationFuture = lookupService
-					.getKvStateLookupInfo(new JobID(), "coffee");
-
-			Await.result(locationFuture, TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (UnknownJobManager ignored) {
-			// Expected
-		}
-
-		// No new messages received
-		assertEquals(0, received.size());
-	}
-
-	/**
-	 * Tests that messages are properly decorated with the leader session ID.
-	 */
-	@Test
-	public void testLeaderSessionIdChange() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			"localhost",
-			HighAvailabilityServices.DEFAULT_LEADER_ID);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		// Create test actors with random leader session IDs
-		KvStateLocation expected1 = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "salt");
-		UUID leaderSessionId1 = UUID.randomUUID();
-		ActorRef testActor1 = LookupResponseActor.create(received, leaderSessionId1, expected1);
-		String testActorAddress1 = AkkaUtils.getAkkaURL(testActorSystem, testActor1);
-
-		KvStateLocation expected2 = new KvStateLocation(new JobID(), new JobVertexID(), 22321, "pepper");
-		UUID leaderSessionId2 = UUID.randomUUID();
-		ActorRef testActor2 = LookupResponseActor.create(received, leaderSessionId1, expected2);
-		String testActorAddress2 = AkkaUtils.getAkkaURL(testActorSystem, testActor2);
-
-		JobID jobId = new JobID();
-
-		//
-		// Notify about first leader
-		//
-		leaderRetrievalService.notifyListener(testActorAddress1, leaderSessionId1);
-
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, "rock"), TIMEOUT);
-		assertEquals(expected1, location);
-
-		assertEquals(1, received.size());
-		verifyLookupMsg(received.poll(), jobId, "rock");
-
-		//
-		// Notify about second leader
-		//
-		leaderRetrievalService.notifyListener(testActorAddress2, leaderSessionId2);
-
-		location = Await.result(lookupService.getKvStateLookupInfo(jobId, "roll"), TIMEOUT);
-		assertEquals(expected2, location);
-
-		assertEquals(1, received.size());
-		verifyLookupMsg(received.poll(), jobId, "roll");
-	}
-
-	/**
-	 * Tests that lookups are retried when no leader notification is available.
-	 */
-	@Test
-	public void testRetryOnUnknownJobManager() throws Exception {
-		final Queue<LookupRetryStrategy> retryStrategies = new LinkedBlockingQueue<>();
-
-		LookupRetryStrategyFactory retryStrategy =
-				new LookupRetryStrategyFactory() {
-					@Override
-					public LookupRetryStrategy createRetryStrategy() {
-						return retryStrategies.poll();
-					}
-				};
-
-		final TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			null,
-			null);
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				retryStrategy);
-
-		lookupService.start();
-
-		//
-		// Test call to retry
-		//
-		final AtomicBoolean hasRetried = new AtomicBoolean();
-		retryStrategies.add(
-				new LookupRetryStrategy() {
-					@Override
-					public FiniteDuration getRetryDelay() {
-						return FiniteDuration.Zero();
-					}
-
-					@Override
-					public boolean tryRetry() {
-						if (hasRetried.compareAndSet(false, true)) {
-							return true;
-						}
-						return false;
-					}
-				});
-
-		Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(new JobID(), "yessir");
-
-		Await.ready(locationFuture, TIMEOUT);
-		assertTrue("Did not retry ", hasRetried.get());
-
-		//
-		// Test leader notification after retry
-		//
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 12122, "garlic");
-		ActorRef testActor = LookupResponseActor.create(received, null, expected);
-		final String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		retryStrategies.add(new LookupRetryStrategy() {
-			@Override
-			public FiniteDuration getRetryDelay() {
-				return FiniteDuration.apply(100, TimeUnit.MILLISECONDS);
-			}
-
-			@Override
-			public boolean tryRetry() {
-				leaderRetrievalService.notifyListener(testActorAddress, HighAvailabilityServices.DEFAULT_LEADER_ID);
-				return true;
-			}
-		});
-
-		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(new JobID(), "yessir"), TIMEOUT);
-		assertEquals(expected, location);
-	}
-
-	@Test
-	public void testUnexpectedResponseType() throws Exception {
-		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
-			"localhost",
-			HighAvailabilityServices.DEFAULT_LEADER_ID);
-		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
-
-		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
-				leaderRetrievalService,
-				testActorSystem,
-				TIMEOUT,
-				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
-
-		lookupService.start();
-
-		// Create test actors with random leader session IDs
-		String expected = "unexpected-response-type";
-		ActorRef testActor = LookupResponseActor.create(received, null, expected);
-		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
-
-		leaderRetrievalService.notifyListener(testActorAddress, null);
-
-		try {
-			Await.result(lookupService.getKvStateLookupInfo(new JobID(), "spicy"), TIMEOUT);
-			fail("Did not throw expected Exception");
-		} catch (Throwable ignored) {
-			// Expected
-		}
-	}
-
-	private static final class LookupResponseActor extends FlinkUntypedActor {
-
-		/** Received lookup messages. */
-		private final Queue<LookupKvStateLocation> receivedLookups;
-
-		/** Responses on KvStateMessage.LookupKvStateLocation messages. */
-		private final Queue<Object> lookupResponses;
-
-		/** The leader session ID. */
-		private UUID leaderSessionId;
-
-		public LookupResponseActor(
-				Queue<LookupKvStateLocation> receivedLookups,
-				UUID leaderSessionId, Object... lookupResponses) {
-
-			this.receivedLookups = Preconditions.checkNotNull(receivedLookups, "Received lookups");
-			this.leaderSessionId = leaderSessionId;
-			this.lookupResponses = new ArrayDeque<>();
-
-			if (lookupResponses != null) {
-				for (Object resp : lookupResponses) {
-					this.lookupResponses.add(resp);
-				}
-			}
-		}
-
-		@Override
-		public void handleMessage(Object message) throws Exception {
-			if (message instanceof LookupKvStateLocation) {
-				// Add to received lookups queue
-				receivedLookups.add((LookupKvStateLocation) message);
-
-				Object msg = lookupResponses.poll();
-				if (msg != null) {
-					if (msg instanceof Throwable) {
-						sender().tell(new Status.Failure((Throwable) msg), self());
-					} else {
-						sender().tell(new Status.Success(msg), self());
-					}
-				}
-			} else if (message instanceof UUID) {
-				this.leaderSessionId = (UUID) message;
-			} else {
-				LOG.debug("Received unhandled message: {}", message);
-			}
-		}
-
-		@Override
-		protected UUID getLeaderSessionID() {
-			return leaderSessionId;
-		}
-
-		private static ActorRef create(
-				Queue<LookupKvStateLocation> receivedLookups,
-				UUID leaderSessionId,
-				Object... lookupResponses) {
-
-			return testActorSystem.actorOf(Props.create(
-					LookupResponseActor.class,
-					receivedLookups,
-					leaderSessionId,
-					lookupResponses));
-		}
-	}
-
-	private static void verifyLookupMsg(
-			LookupKvStateLocation lookUpMsg,
-			JobID expectedJobId,
-			String expectedName) {
-
-		assertNotNull(lookUpMsg);
-		assertEquals(expectedJobId, lookUpMsg.getJobId());
-		assertEquals(expectedName, lookUpMsg.getRegistrationName());
-	}
-
-}


[02/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java
deleted file mode 100644
index afd9e46..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java
+++ /dev/null
@@ -1,449 +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.query;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.KvStateClient;
-import org.apache.flink.runtime.query.netty.KvStateServer;
-import org.apache.flink.runtime.query.netty.UnknownKvStateID;
-import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.heap.HeapValueState;
-import org.apache.flink.runtime.state.heap.NestedMapsStateTable;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.util.MathUtils;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link QueryableStateClient}.
- */
-public class QueryableStateClientTest {
-
-	private static final ActorSystem testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
-
-	private static final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS);
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (testActorSystem != null) {
-			testActorSystem.shutdown();
-		}
-	}
-
-	/**
-	 * All failures should lead to a retry with a forced location lookup.
-	 *
-	 * <p>UnknownKvStateID, UnknownKvStateKeyGroupLocation, UnknownKvStateLocation,
-	 * ConnectException are checked explicitly as these indicate out-of-sync
-	 * KvStateLocation.
-	 */
-	@Test
-	public void testForceLookupOnOutdatedLocation() throws Exception {
-		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-		KvStateClient networkClient = mock(KvStateClient.class);
-
-		QueryableStateClient client = new QueryableStateClient(
-				lookupService,
-				networkClient,
-				testActorSystem.dispatcher());
-
-		try {
-			JobID jobId = new JobID();
-			int numKeyGroups = 4;
-
-			//
-			// UnknownKvStateLocation
-			//
-			String query1 = "lucky";
-
-			Future<KvStateLocation> unknownKvStateLocation = Futures.failed(
-					new UnknownKvStateLocation(query1));
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query1)))
-					.thenReturn(unknownKvStateLocation);
-
-			Future<Integer> result = client.getKvState(
-					jobId,
-					query1,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateLocation exception");
-			} catch (UnknownKvStateLocation ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query1));
-
-			//
-			// UnknownKvStateKeyGroupLocation
-			//
-			String query2 = "unlucky";
-
-			Future<KvStateLocation> unknownKeyGroupLocation = Futures.successful(
-					new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query2));
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query2)))
-					.thenReturn(unknownKeyGroupLocation);
-
-			result = client.getKvState(
-					jobId,
-					query2,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateKeyGroupLocation exception");
-			} catch (UnknownKvStateKeyGroupLocation ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query2));
-
-			//
-			// UnknownKvStateID
-			//
-			String query3 = "water";
-			KvStateID kvStateId = new KvStateID();
-			Future<byte[]> unknownKvStateId = Futures.failed(new UnknownKvStateID(kvStateId));
-
-			KvStateServerAddress serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 12323);
-			KvStateLocation location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query3);
-			for (int i = 0; i < numKeyGroups; i++) {
-				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
-			}
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query3)))
-					.thenReturn(Futures.successful(location));
-
-			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
-					.thenReturn(unknownKvStateId);
-
-			result = client.getKvState(
-					jobId,
-					query3,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected UnknownKvStateID exception");
-			} catch (UnknownKvStateID ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query3));
-
-			//
-			// ConnectException
-			//
-			String query4 = "space";
-			Future<byte[]> connectException = Futures.failed(new ConnectException());
-			kvStateId = new KvStateID();
-
-			serverAddress = new KvStateServerAddress(InetAddress.getLocalHost(), 11123);
-			location = new KvStateLocation(jobId, new JobVertexID(), numKeyGroups, query4);
-			for (int i = 0; i < numKeyGroups; i++) {
-				location.registerKvState(new KeyGroupRange(i, i), kvStateId, serverAddress);
-			}
-
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query4)))
-					.thenReturn(Futures.successful(location));
-
-			when(networkClient.getKvState(eq(serverAddress), eq(kvStateId), any(byte[].class)))
-					.thenReturn(connectException);
-
-			result = client.getKvState(
-					jobId,
-					query4,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			try {
-				Await.result(result, timeout);
-				fail("Did not throw expected ConnectException exception");
-			} catch (ConnectException ignored) {
-				// Expected
-			}
-
-			verify(lookupService, times(2)).getKvStateLookupInfo(eq(jobId), eq(query4));
-
-			//
-			// Other Exceptions don't lead to a retry no retry
-			//
-			String query5 = "universe";
-			Future<KvStateLocation> exception = Futures.failed(new RuntimeException("Test exception"));
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq(query5)))
-					.thenReturn(exception);
-
-			client.getKvState(
-					jobId,
-					query5,
-					0,
-					BasicTypeInfo.INT_TYPE_INFO,
-					new ValueStateDescriptor<>("test", IntSerializer.INSTANCE));
-
-			verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId), eq(query5));
-		} finally {
-			client.shutDown();
-		}
-	}
-
-	/**
-	 * Tests queries against multiple servers.
-	 *
-	 * <p>The servers are populated with different keys and the client queries
-	 * all available keys from all servers.
-	 */
-	@Test
-	public void testIntegrationWithKvStateServer() throws Exception {
-		// Config
-		int numServers = 2;
-		int numKeys = 1024;
-		int numKeyGroups = 1;
-
-		JobID jobId = new JobID();
-		JobVertexID jobVertexId = new JobVertexID();
-
-		KvStateServer[] servers = new KvStateServer[numServers];
-		AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
-
-		QueryableStateClient client = null;
-		KvStateClient networkClient = null;
-		AtomicKvStateRequestStats networkClientStats = new AtomicKvStateRequestStats();
-
-		MemoryStateBackend backend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-
-		AbstractKeyedStateBackend<Integer> keyedStateBackend = backend.createKeyedStateBackend(dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				new KvStateRegistry().createTaskRegistry(new JobID(), new JobVertexID()));
-
-		try {
-			KvStateRegistry[] registries = new KvStateRegistry[numServers];
-			KvStateID[] kvStateIds = new KvStateID[numServers];
-			List<HeapValueState<Integer, VoidNamespace, Integer>> kvStates = new ArrayList<>();
-
-			// Start the servers
-			for (int i = 0; i < numServers; i++) {
-				registries[i] = new KvStateRegistry();
-				serverStats[i] = new AtomicKvStateRequestStats();
-				servers[i] = new KvStateServer(InetAddress.getLocalHost(), 0, 1, 1, registries[i], serverStats[i]);
-				servers[i].start();
-				ValueStateDescriptor<Integer> descriptor =
-						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-
-				RegisteredKeyedBackendStateMetaInfo<VoidNamespace, Integer> registeredKeyedBackendStateMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
-						descriptor.getType(),
-						descriptor.getName(),
-						VoidNamespaceSerializer.INSTANCE,
-						IntSerializer.INSTANCE);
-
-				// Register state
-				HeapValueState<Integer, VoidNamespace, Integer> kvState = new HeapValueState<>(
-						descriptor,
-						new NestedMapsStateTable<>(keyedStateBackend, registeredKeyedBackendStateMetaInfo),
-						IntSerializer.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE);
-
-				kvStates.add(kvState);
-
-				kvStateIds[i] = registries[i].registerKvState(
-						jobId,
-						new JobVertexID(),
-						new KeyGroupRange(i, i),
-						"choco",
-						kvState);
-			}
-
-			int[] expectedRequests = new int[numServers];
-
-			for (int key = 0; key < numKeys; key++) {
-				int targetKeyGroupIndex = MathUtils.murmurHash(key) % numServers;
-				expectedRequests[targetKeyGroupIndex]++;
-
-				HeapValueState<Integer, VoidNamespace, Integer> kvState = kvStates.get(targetKeyGroupIndex);
-
-				keyedStateBackend.setCurrentKey(key);
-				kvState.setCurrentNamespace(VoidNamespace.INSTANCE);
-				kvState.update(1337 + key);
-			}
-
-			// Location lookup service
-			KvStateLocation location = new KvStateLocation(jobId, jobVertexId, numServers, "choco");
-			for (int keyGroupIndex = 0; keyGroupIndex < numServers; keyGroupIndex++) {
-				location.registerKvState(new KeyGroupRange(keyGroupIndex, keyGroupIndex), kvStateIds[keyGroupIndex], servers[keyGroupIndex].getAddress());
-			}
-
-			KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-			when(lookupService.getKvStateLookupInfo(eq(jobId), eq("choco")))
-					.thenReturn(Futures.successful(location));
-
-			// The client
-			networkClient = new KvStateClient(1, networkClientStats);
-
-			client = new QueryableStateClient(lookupService, networkClient, testActorSystem.dispatcher());
-
-			// Send all queries
-			List<Future<Integer>> futures = new ArrayList<>(numKeys);
-			for (int key = 0; key < numKeys; key++) {
-				ValueStateDescriptor<Integer> descriptor =
-						new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-				futures.add(client.getKvState(
-						jobId,
-						"choco",
-						key,
-						BasicTypeInfo.INT_TYPE_INFO,
-						descriptor));
-			}
-
-			// Verify results
-			Future<Iterable<Integer>> future = Futures.sequence(futures, testActorSystem.dispatcher());
-			Iterable<Integer> results = Await.result(future, timeout);
-
-			int index = 0;
-			for (int buffer : results) {
-				assertEquals(1337 + index, buffer);
-				index++;
-			}
-
-			// Verify requests
-			for (int i = 0; i < numServers; i++) {
-				int numRetries = 10;
-				for (int retry = 0; retry < numRetries; retry++) {
-					try {
-						assertEquals("Unexpected number of requests", expectedRequests[i], serverStats[i].getNumRequests());
-						assertEquals("Unexpected success requests", expectedRequests[i], serverStats[i].getNumSuccessful());
-						assertEquals("Unexpected failed requests", 0, serverStats[i].getNumFailed());
-						break;
-					} catch (Throwable t) {
-						// Retry
-						if (retry == numRetries - 1) {
-							throw t;
-						} else {
-							Thread.sleep(100);
-						}
-					}
-				}
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (networkClient != null) {
-				networkClient.shutDown();
-			}
-
-			for (KvStateServer server : servers) {
-				if (server != null) {
-					server.shutDown();
-				}
-			}
-		}
-	}
-
-	/**
-	 * Tests that the QueryableState client correctly caches location lookups
-	 * keyed by both job and name. This test is mainly due to a previous bug due
-	 * to which cache entries were by name only. This is a problem, because the
-	 * same client can be used to query multiple jobs.
-	 */
-	@Test
-	public void testLookupMultipleJobIds() throws Exception {
-		String name = "unique-per-job";
-
-		// Exact contents don't matter here
-		KvStateLocation location = new KvStateLocation(new JobID(), new JobVertexID(), 1, name);
-		location.registerKvState(new KeyGroupRange(0, 0), new KvStateID(), new KvStateServerAddress(InetAddress.getLocalHost(), 892));
-
-		JobID jobId1 = new JobID();
-		JobID jobId2 = new JobID();
-
-		KvStateLocationLookupService lookupService = mock(KvStateLocationLookupService.class);
-
-		when(lookupService.getKvStateLookupInfo(any(JobID.class), anyString()))
-				.thenReturn(Futures.successful(location));
-
-		KvStateClient networkClient = mock(KvStateClient.class);
-		when(networkClient.getKvState(any(KvStateServerAddress.class), any(KvStateID.class), any(byte[].class)))
-				.thenReturn(Futures.successful(new byte[0]));
-
-		QueryableStateClient client = new QueryableStateClient(
-				lookupService,
-				networkClient,
-				testActorSystem.dispatcher());
-
-		ValueStateDescriptor<Integer> stateDesc = new ValueStateDescriptor<>("test", IntSerializer.INSTANCE);
-
-		// Query ies with same name, but different job IDs should lead to a
-		// single lookup per query and job ID.
-		client.getKvState(jobId1, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
-		client.getKvState(jobId2, name, 0, BasicTypeInfo.INT_TYPE_INFO, stateDesc);
-
-		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId1), eq(name));
-		verify(lookupService, times(1)).getKvStateLookupInfo(eq(jobId2), eq(name));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
deleted file mode 100644
index 1e41236..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientHandlerTest.java
+++ /dev/null
@@ -1,115 +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.query.netty;
-
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
-
-import org.junit.Test;
-
-import java.nio.channels.ClosedChannelException;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-/**
- * Tests for {@link KvStateClientHandler}.
- */
-public class KvStateClientHandlerTest {
-
-	/**
-	 * Tests that on reads the expected callback methods are called and read
-	 * buffers are recycled.
-	 */
-	@Test
-	public void testReadCallbacksAndBufferRecycling() throws Exception {
-		KvStateClientHandlerCallback callback = mock(KvStateClientHandlerCallback.class);
-
-		EmbeddedChannel channel = new EmbeddedChannel(new KvStateClientHandler(callback));
-
-		//
-		// Request success
-		//
-		ByteBuf buf = KvStateRequestSerializer.serializeKvStateRequestResult(
-				channel.alloc(),
-				1222112277,
-				new byte[0]);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify callback
-		channel.writeInbound(buf);
-		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(byte[].class));
-		assertEquals("Buffer not recycled", 0, buf.refCnt());
-
-		//
-		// Request failure
-		//
-		buf = KvStateRequestSerializer.serializeKvStateRequestFailure(
-				channel.alloc(),
-				1222112278,
-				new RuntimeException("Expected test Exception"));
-		buf.skipBytes(4); // skip frame length
-
-		// Verify callback
-		channel.writeInbound(buf);
-		verify(callback, times(1)).onRequestFailure(eq(1222112278L), any(RuntimeException.class));
-		assertEquals("Buffer not recycled", 0, buf.refCnt());
-
-		//
-		// Server failure
-		//
-		buf = KvStateRequestSerializer.serializeServerFailure(
-				channel.alloc(),
-				new RuntimeException("Expected test Exception"));
-		buf.skipBytes(4); // skip frame length
-
-		// Verify callback
-		channel.writeInbound(buf);
-		verify(callback, times(1)).onFailure(any(RuntimeException.class));
-
-		//
-		// Unexpected messages
-		//
-		buf = channel.alloc().buffer(4).writeInt(1223823);
-
-		// Verify callback
-		channel.writeInbound(buf);
-		verify(callback, times(2)).onFailure(any(IllegalStateException.class));
-		assertEquals("Buffer not recycled", 0, buf.refCnt());
-
-		//
-		// Exception caught
-		//
-		channel.pipeline().fireExceptionCaught(new RuntimeException("Expected test Exception"));
-		verify(callback, times(3)).onFailure(any(RuntimeException.class));
-
-		//
-		// Channel inactive
-		//
-		channel.pipeline().fireChannelInactive();
-		verify(callback, times(4)).onFailure(any(ClosedChannelException.class));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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
deleted file mode 100644
index 6b21487..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateClientTest.java
+++ /dev/null
@@ -1,747 +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.query.netty;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-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.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-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.InternalKvState;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.util.NetUtils;
-
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link KvStateClient}.
- */
-public class KvStateClientTest {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientTest.class);
-
-	// Thread pool for client bootstrap (shared between tests)
-	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
-
-	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(100, TimeUnit.SECONDS);
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (NIO_GROUP != null) {
-			NIO_GROUP.shutdownGracefully();
-		}
-	}
-
-	/**
-	 * Tests simple queries, of which half succeed and half fail.
-	 */
-	@Test
-	public void testSimpleRequests() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			// Random result
-			final byte[] expected = new byte[1024];
-			ThreadLocalRandom.current().nextBytes(expected);
-
-			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.add((ByteBuf) msg);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			List<Future<byte[]>> futures = new ArrayList<>();
-
-			int numQueries = 1024;
-
-			for (int i = 0; i < numQueries; i++) {
-				futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-			}
-
-			// Respond to messages
-			Exception testException = new RuntimeException("Expected test Exception");
-
-			for (int i = 0; i < numQueries; i++) {
-				ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-				assertNotNull("Receive timed out", buf);
-
-				Channel ch = channel.get();
-				assertNotNull("Channel not active", ch);
-
-				assertEquals(KvStateRequestType.REQUEST, KvStateRequestSerializer.deserializeHeader(buf));
-				KvStateRequest request = KvStateRequestSerializer.deserializeKvStateRequest(buf);
-
-				buf.release();
-
-				if (i % 2 == 0) {
-					ByteBuf response = KvStateRequestSerializer.serializeKvStateRequestResult(
-							serverChannel.alloc(),
-							request.getRequestId(),
-							expected);
-
-					ch.writeAndFlush(response);
-				} else {
-					ByteBuf response = KvStateRequestSerializer.serializeKvStateRequestFailure(
-							serverChannel.alloc(),
-							request.getRequestId(),
-							testException);
-
-					ch.writeAndFlush(response);
-				}
-			}
-
-			for (int i = 0; i < numQueries; i++) {
-				if (i % 2 == 0) {
-					byte[] serializedResult = Await.result(futures.get(i), deadline.timeLeft());
-					assertArrayEquals(expected, serializedResult);
-				} else {
-					try {
-						Await.result(futures.get(i), deadline.timeLeft());
-						fail("Did not throw expected Exception");
-					} catch (RuntimeException ignored) {
-						// Expected
-					}
-				}
-			}
-
-			assertEquals(numQueries, stats.getNumRequests());
-			int expectedRequests = numQueries / 2;
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != expectedRequests ||
-					stats.getNumFailed() != expectedRequests)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(expectedRequests, stats.getNumSuccessful());
-			assertEquals(expectedRequests, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a request to an unavailable host is failed with ConnectException.
-	 */
-	@Test
-	public void testRequestUnavailableHost() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-		KvStateClient client = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			int availablePort = NetUtils.getAvailablePort();
-
-			KvStateServerAddress serverAddress = new KvStateServerAddress(
-					InetAddress.getLocalHost(),
-					availablePort);
-
-			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
-
-			try {
-				Await.result(future, deadline.timeLeft());
-				fail("Did not throw expected ConnectException");
-			} catch (ConnectException ignored) {
-				// Expected
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Multiple threads concurrently fire queries.
-	 */
-	@Test
-	public void testConcurrentQueries() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		ExecutorService executor = null;
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		final byte[] serializedResult = new byte[1024];
-		ThreadLocalRandom.current().nextBytes(serializedResult);
-
-		try {
-			int numQueryTasks = 4;
-			final int numQueriesPerTask = 1024;
-
-			executor = Executors.newFixedThreadPool(numQueryTasks);
-
-			client = new KvStateClient(1, stats);
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					ByteBuf buf = (ByteBuf) msg;
-					assertEquals(KvStateRequestType.REQUEST, KvStateRequestSerializer.deserializeHeader(buf));
-					KvStateRequest request = KvStateRequestSerializer.deserializeKvStateRequest(buf);
-
-					buf.release();
-
-					ByteBuf response = KvStateRequestSerializer.serializeKvStateRequestResult(
-							ctx.alloc(),
-							request.getRequestId(),
-							serializedResult);
-
-					ctx.channel().writeAndFlush(response);
-				}
-			});
-
-			final KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			final KvStateClient finalClient = client;
-			Callable<List<Future<byte[]>>> queryTask = new Callable<List<Future<byte[]>>>() {
-				@Override
-				public List<Future<byte[]>> call() throws Exception {
-					List<Future<byte[]>> results = new ArrayList<>(numQueriesPerTask);
-
-					for (int i = 0; i < numQueriesPerTask; i++) {
-						results.add(finalClient.getKvState(
-								serverAddress,
-								new KvStateID(),
-								new byte[0]));
-					}
-
-					return results;
-				}
-			};
-
-			// Submit query tasks
-			List<java.util.concurrent.Future<List<Future<byte[]>>>> futures = new ArrayList<>();
-			for (int i = 0; i < numQueryTasks; i++) {
-				futures.add(executor.submit(queryTask));
-			}
-
-			// Verify results
-			for (java.util.concurrent.Future<List<Future<byte[]>>> future : futures) {
-				List<Future<byte[]>> results = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-				for (Future<byte[]> result : results) {
-					byte[] actual = Await.result(result, deadline.timeLeft());
-					assertArrayEquals(serializedResult, actual);
-				}
-			}
-
-			int totalQueries = numQueryTasks * numQueriesPerTask;
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && stats.getNumSuccessful() != totalQueries) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(totalQueries, stats.getNumRequests());
-			assertEquals(totalQueries, stats.getNumSuccessful());
-		} finally {
-			if (executor != null) {
-				executor.shutdown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			if (client != null) {
-				client.shutDown();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a server failure closes the connection and removes it from
-	 * the established connections.
-	 */
-	@Test
-	public void testFailureClosesChannel() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.add((ByteBuf) msg);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			// Requests
-			List<Future<byte[]>> futures = new ArrayList<>();
-			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-
-			ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			assertNotNull("Receive timed out", buf);
-			buf.release();
-
-			buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			assertNotNull("Receive timed out", buf);
-			buf.release();
-
-			assertEquals(1, stats.getNumConnections());
-
-			Channel ch = channel.get();
-			assertNotNull("Channel not active", ch);
-
-			// Respond with failure
-			ch.writeAndFlush(KvStateRequestSerializer.serializeServerFailure(
-					serverChannel.alloc(),
-					new RuntimeException("Expected test server failure")));
-
-			try {
-				Await.result(futures.remove(0), deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (RuntimeException ignored) {
-				// Expected
-			}
-
-			try {
-				Await.result(futures.remove(0), deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (RuntimeException ignored) {
-				// Expected
-			}
-
-			assertEquals(0, stats.getNumConnections());
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
-					stats.getNumFailed() != 2)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(2, stats.getNumRequests());
-			assertEquals(0, stats.getNumSuccessful());
-			assertEquals(2, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a server channel close, closes the connection and removes it
-	 * from the established connections.
-	 */
-	@Test
-	public void testServerClosesChannel() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			final AtomicBoolean received = new AtomicBoolean();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.set(true);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			// Requests
-			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
-
-			while (!received.get() && deadline.hasTimeLeft()) {
-				Thread.sleep(50);
-			}
-			assertTrue("Receive timed out", received.get());
-
-			assertEquals(1, stats.getNumConnections());
-
-			channel.get().close().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-			try {
-				Await.result(future, deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (ClosedChannelException ignored) {
-				// Expected
-			}
-
-			assertEquals(0, stats.getNumConnections());
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
-					stats.getNumFailed() != 1)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(1, stats.getNumRequests());
-			assertEquals(0, stats.getNumSuccessful());
-			assertEquals(1, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests multiple clients querying multiple servers until 100k queries have
-	 * been processed. At this point, the client is shut down and its verified
-	 * that all ongoing requests are failed.
-	 */
-	@Test
-	public void testClientServerIntegration() throws Exception {
-		// Config
-		final int numServers = 2;
-		final int numServerEventLoopThreads = 2;
-		final int numServerQueryThreads = 2;
-
-		final int numClientEventLoopThreads = 4;
-		final int numClientsTasks = 8;
-
-		final int batchSize = 16;
-
-		final int numKeyGroups = 1;
-
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		KvStateRegistry dummyRegistry = new KvStateRegistry();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(dummyRegistry);
-
-		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				dummyRegistry.createTaskRegistry(new JobID(), new JobVertexID()));
-
-		final FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
-
-		AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		ExecutorService clientTaskExecutor = null;
-		final KvStateServer[] server = new KvStateServer[numServers];
-
-		try {
-			client = new KvStateClient(numClientEventLoopThreads, clientStats);
-			clientTaskExecutor = Executors.newFixedThreadPool(numClientsTasks);
-
-			// Create state
-			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-			desc.setQueryable("any");
-
-			// Create servers
-			KvStateRegistry[] registry = new KvStateRegistry[numServers];
-			AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
-			final KvStateID[] ids = new KvStateID[numServers];
-
-			for (int i = 0; i < numServers; i++) {
-				registry[i] = new KvStateRegistry();
-				serverStats[i] = new AtomicKvStateRequestStats();
-				server[i] = new KvStateServer(
-						InetAddress.getLocalHost(),
-						0,
-						numServerEventLoopThreads,
-						numServerQueryThreads,
-						registry[i],
-						serverStats[i]);
-
-				server[i].start();
-
-				backend.setCurrentKey(1010 + i);
-
-				// Value per server
-				ValueState<Integer> state = backend.getPartitionedState(VoidNamespace.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE,
-						desc);
-
-				state.update(201 + i);
-
-				// we know it must be a KvStat but this is not exposed to the user via 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);
-			}
-
-			final KvStateClient finalClient = client;
-			Callable<Void> queryTask = new Callable<Void>() {
-				@Override
-				public Void call() throws Exception {
-					while (true) {
-						if (Thread.interrupted()) {
-							throw new InterruptedException();
-						}
-
-						// Random server permutation
-						List<Integer> random = new ArrayList<>();
-						for (int j = 0; j < batchSize; j++) {
-							random.add(j);
-						}
-						Collections.shuffle(random);
-
-						// Dispatch queries
-						List<Future<byte[]>> futures = new ArrayList<>(batchSize);
-
-						for (int j = 0; j < batchSize; j++) {
-							int targetServer = random.get(j) % numServers;
-
-							byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-									1010 + targetServer,
-									IntSerializer.INSTANCE,
-									VoidNamespace.INSTANCE,
-									VoidNamespaceSerializer.INSTANCE);
-
-							futures.add(finalClient.getKvState(
-									server[targetServer].getAddress(),
-									ids[targetServer],
-									serializedKeyAndNamespace));
-						}
-
-						// Verify results
-						for (int j = 0; j < batchSize; j++) {
-							int targetServer = random.get(j) % numServers;
-
-							Future<byte[]> future = futures.get(j);
-							byte[] buf = Await.result(future, timeout);
-							int value = KvStateRequestSerializer.deserializeValue(buf, IntSerializer.INSTANCE);
-							assertEquals(201 + targetServer, value);
-						}
-					}
-				}
-			};
-
-			// Submit tasks
-			List<java.util.concurrent.Future<Void>> taskFutures = new ArrayList<>();
-			for (int i = 0; i < numClientsTasks; i++) {
-				taskFutures.add(clientTaskExecutor.submit(queryTask));
-			}
-
-			long numRequests;
-			while ((numRequests = clientStats.getNumRequests()) < 100_000) {
-				Thread.sleep(100);
-				LOG.info("Number of requests {}/100_000", numRequests);
-			}
-
-			// Shut down
-			client.shutDown();
-
-			for (java.util.concurrent.Future<Void> future : taskFutures) {
-				try {
-					future.get();
-					fail("Did not throw expected Exception after shut down");
-				} catch (ExecutionException t) {
-					if (t.getCause() instanceof ClosedChannelException ||
-							t.getCause() instanceof IllegalStateException) {
-						// Expected
-					} else {
-						t.printStackTrace();
-						fail("Failed with unexpected Exception type: " + t.getClass().getName());
-					}
-				}
-			}
-
-			assertEquals("Connection leak (client)", 0, clientStats.getNumConnections());
-			for (int i = 0; i < numServers; i++) {
-				boolean success = false;
-				int numRetries = 0;
-				while (!success) {
-					try {
-						assertEquals("Connection leak (server)", 0, serverStats[i].getNumConnections());
-						success = true;
-					} catch (Throwable t) {
-						if (numRetries < 10) {
-							LOG.info("Retrying connection leak check (server)");
-							Thread.sleep((numRetries + 1) * 50);
-							numRetries++;
-						} else {
-							throw t;
-						}
-					}
-				}
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			for (int i = 0; i < numServers; i++) {
-				if (server[i] != null) {
-					server[i].shutDown();
-				}
-			}
-
-			if (clientTaskExecutor != null) {
-				clientTaskExecutor.shutdown();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private Channel createServerChannel(final ChannelHandler... handlers) throws UnknownHostException, InterruptedException {
-		ServerBootstrap bootstrap = new ServerBootstrap()
-				// Bind address and port
-				.localAddress(InetAddress.getLocalHost(), 0)
-				// NIO server channels
-				.group(NIO_GROUP)
-				.channel(NioServerSocketChannel.class)
-				// See initializer for pipeline details
-				.childHandler(new ChannelInitializer<SocketChannel>() {
-					@Override
-					protected void initChannel(SocketChannel ch) throws Exception {
-						ch.pipeline()
-								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-								.addLast(handlers);
-					}
-				});
-
-		return bootstrap.bind().sync().channel();
-	}
-
-	private KvStateServerAddress getKvStateServerAddress(Channel serverChannel) {
-		InetSocketAddress localAddress = (InetSocketAddress) serverChannel.localAddress();
-
-		return new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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
deleted file mode 100644
index 4914ff7..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerHandlerTest.java
+++ /dev/null
@@ -1,721 +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.query.netty;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateRegistryListener;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestFailure;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestResult;
-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.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyedStateBackend;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.internal.InternalKvState;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link KvStateServerHandler}.
- */
-public class KvStateServerHandlerTest extends TestLogger {
-
-	/** Shared Thread pool for query execution. */
-	private static final ExecutorService TEST_THREAD_POOL = Executors.newSingleThreadExecutor();
-
-	private static final int READ_TIMEOUT_MILLIS = 10000;
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (TEST_THREAD_POOL != null) {
-			TEST_THREAD_POOL.shutdown();
-		}
-	}
-
-	/**
-	 * Tests a simple successful query via an EmbeddedChannel.
-	 */
-	@Test
-	public void testSimpleQuery() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		// Register state
-		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-		desc.setQueryable("vanilla");
-
-		int numKeyGroups = 1;
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(registry);
-		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
-
-		final TestRegistryListener registryListener = new TestRegistryListener();
-		registry.registerListener(registryListener);
-
-		// Update the KvState and request it
-		int expectedValue = 712828289;
-
-		int key = 99812822;
-		backend.setCurrentKey(key);
-		ValueState<Integer> state = backend.getPartitionedState(
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE,
-				desc);
-
-		state.update(expectedValue);
-
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-				key,
-				IntSerializer.INSTANCE,
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE);
-
-		long requestId = Integer.MAX_VALUE + 182828L;
-
-		assertTrue(registryListener.registrationName.equals("vanilla"));
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_RESULT, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestResult response = KvStateRequestSerializer.deserializeKvStateRequestResult(buf);
-
-		assertEquals(requestId, response.getRequestId());
-
-		int actualValue = KvStateRequestSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
-		assertEquals(expectedValue, actualValue);
-
-		assertEquals(stats.toString(), 1, stats.getNumRequests());
-
-		// Wait for async successful request report
-		long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
-		while (stats.getNumSuccessful() != 1 && System.nanoTime() <= deadline) {
-			Thread.sleep(10);
-		}
-
-		assertEquals(stats.toString(), 1, stats.getNumSuccessful());
-	}
-
-	/**
-	 * Tests the failure response with {@link UnknownKvStateID} as cause on
-	 * queries for unregistered KvStateIDs.
-	 */
-	@Test
-	public void testQueryUnknownKvStateID() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		long requestId = Integer.MAX_VALUE + 182828L;
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				new KvStateID(),
-				new byte[0]);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-
-		assertEquals(requestId, response.getRequestId());
-
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateID);
-
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests the failure response with {@link UnknownKeyOrNamespace} as cause
-	 * on queries for non-existing keys.
-	 */
-	@Test
-	public void testQueryUnknownKey() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		int numKeyGroups = 1;
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(registry);
-		KeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
-
-		final TestRegistryListener registryListener = new TestRegistryListener();
-		registry.registerListener(registryListener);
-
-		// Register state
-		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-		desc.setQueryable("vanilla");
-
-		backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);
-
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-				1238283,
-				IntSerializer.INSTANCE,
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE);
-
-		long requestId = Integer.MAX_VALUE + 22982L;
-
-		assertTrue(registryListener.registrationName.equals("vanilla"));
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-
-		assertEquals(requestId, response.getRequestId());
-
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespace);
-
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests the failure response on a failure on the {@link InternalKvState#getSerializedValue(byte[])}
-	 * call.
-	 */
-	@Test
-	public void testFailureOnGetSerializedValue() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		// Failing KvState
-		InternalKvState<?> kvState = mock(InternalKvState.class);
-		when(kvState.getSerializedValue(any(byte[].class)))
-				.thenThrow(new RuntimeException("Expected test Exception"));
-
-		KvStateID kvStateId = registry.registerKvState(
-				new JobID(),
-				new JobVertexID(),
-				new KeyGroupRange(0, 0),
-				"vanilla",
-				kvState);
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				kvStateId,
-				new byte[0]);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-
-		assertTrue(response.getCause().getMessage().contains("Expected test Exception"));
-
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests that the channel is closed if an Exception reaches the channel
-	 * handler.
-	 */
-	@Test
-	public void testCloseChannelOnExceptionCaught() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(handler);
-
-		channel.pipeline().fireExceptionCaught(new RuntimeException("Expected test Exception"));
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.SERVER_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		Throwable response = KvStateRequestSerializer.deserializeServerFailure(buf);
-
-		assertTrue(response.getMessage().contains("Expected test Exception"));
-
-		channel.closeFuture().await(READ_TIMEOUT_MILLIS);
-		assertFalse(channel.isActive());
-	}
-
-	/**
-	 * Tests the failure response on a rejected execution, because the query
-	 * executor has been closed.
-	 */
-	@Test
-	public void testQueryExecutorShutDown() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		ExecutorService closedExecutor = Executors.newSingleThreadExecutor();
-		closedExecutor.shutdown();
-		assertTrue(closedExecutor.isShutdown());
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, closedExecutor, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		int numKeyGroups = 1;
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(registry);
-		KeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
-
-		final TestRegistryListener registryListener = new TestRegistryListener();
-		registry.registerListener(registryListener);
-
-		// Register state
-		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-		desc.setQueryable("vanilla");
-
-		backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, desc);
-
-		assertTrue(registryListener.registrationName.equals("vanilla"));
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				registryListener.kvStateId,
-				new byte[0]);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-
-		assertTrue(response.getCause().getMessage().contains("RejectedExecutionException"));
-
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests response on unexpected messages.
-	 */
-	@Test
-	public void testUnexpectedMessage() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		// Write the request and wait for the response
-		ByteBuf unexpectedMessage = Unpooled.buffer(8);
-		unexpectedMessage.writeInt(4);
-		unexpectedMessage.writeInt(123238213);
-
-		channel.writeInbound(unexpectedMessage);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.SERVER_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		Throwable response = KvStateRequestSerializer.deserializeServerFailure(buf);
-
-		assertEquals(0, stats.getNumRequests());
-		assertEquals(0, stats.getNumFailed());
-
-		unexpectedMessage = KvStateRequestSerializer.serializeKvStateRequestResult(
-				channel.alloc(),
-				192,
-				new byte[0]);
-
-		channel.writeInbound(unexpectedMessage);
-
-		buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.SERVER_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		response = KvStateRequestSerializer.deserializeServerFailure(buf);
-
-		assertTrue("Unexpected failure cause " + response.getClass().getName(), response instanceof IllegalArgumentException);
-
-		assertEquals(0, stats.getNumRequests());
-		assertEquals(0, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests that incoming buffer instances are recycled.
-	 */
-	@Test
-	public void testIncomingBufferIsRecycled() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				new KvStateID(),
-				new byte[0]);
-
-		assertEquals(1, request.refCnt());
-
-		// Write regular request
-		channel.writeInbound(request);
-		assertEquals("Buffer not recycled", 0, request.refCnt());
-
-		// Write unexpected msg
-		ByteBuf unexpected = channel.alloc().buffer(8);
-		unexpected.writeInt(4);
-		unexpected.writeInt(4);
-
-		assertEquals(1, unexpected.refCnt());
-
-		channel.writeInbound(unexpected);
-		assertEquals("Buffer not recycled", 0, unexpected.refCnt());
-	}
-
-	/**
-	 * Tests the failure response if the serializers don't match.
-	 */
-	@Test
-	public void testSerializerMismatch() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		int numKeyGroups = 1;
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(registry);
-		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
-
-		final TestRegistryListener registryListener = new TestRegistryListener();
-		registry.registerListener(registryListener);
-
-		// Register state
-		ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-		desc.setQueryable("vanilla");
-
-		ValueState<Integer> state = backend.getPartitionedState(
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE,
-				desc);
-
-		int key = 99812822;
-
-		// Update the KvState
-		backend.setCurrentKey(key);
-		state.update(712828289);
-
-		byte[] wrongKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-				"wrong-key-type",
-				StringSerializer.INSTANCE,
-				"wrong-namespace-type",
-				StringSerializer.INSTANCE);
-
-		byte[] wrongNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-				key,
-				IntSerializer.INSTANCE,
-				"wrong-namespace-type",
-				StringSerializer.INSTANCE);
-
-		assertTrue(registryListener.registrationName.equals("vanilla"));
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				182828,
-				registryListener.kvStateId,
-				wrongKeyAndNamespace);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(182828, response.getRequestId());
-		assertTrue(response.getCause().getMessage().contains("IOException"));
-
-		// Repeat with wrong namespace only
-		request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				182829,
-				registryListener.kvStateId,
-				wrongNamespace);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		buf = (ByteBuf) readInboundBlocking(channel);
-		buf.skipBytes(4); // skip frame length
-
-		// Verify the response
-		assertEquals(KvStateRequestType.REQUEST_FAILURE, KvStateRequestSerializer.deserializeHeader(buf));
-		response = KvStateRequestSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(182829, response.getRequestId());
-		assertTrue(response.getCause().getMessage().contains("IOException"));
-
-		assertEquals(2, stats.getNumRequests());
-		assertEquals(2, stats.getNumFailed());
-	}
-
-	/**
-	 * Tests that large responses are chunked.
-	 */
-	@Test
-	public void testChunkedResponse() throws Exception {
-		KvStateRegistry registry = new KvStateRegistry();
-		KvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
-		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
-
-		int numKeyGroups = 1;
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(registry);
-		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				registry.createTaskRegistry(dummyEnv.getJobID(), dummyEnv.getJobVertexId()));
-
-		final TestRegistryListener registryListener = new TestRegistryListener();
-		registry.registerListener(registryListener);
-
-		// Register state
-		ValueStateDescriptor<byte[]> desc = new ValueStateDescriptor<>("any", BytePrimitiveArraySerializer.INSTANCE);
-		desc.setQueryable("vanilla");
-
-		ValueState<byte[]> state = backend.getPartitionedState(
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE,
-				desc);
-
-		// Update KvState
-		byte[] bytes = new byte[2 * channel.config().getWriteBufferHighWaterMark()];
-
-		byte current = 0;
-		for (int i = 0; i < bytes.length; i++) {
-			bytes[i] = current++;
-		}
-
-		int key = 99812822;
-		backend.setCurrentKey(key);
-		state.update(bytes);
-
-		// Request
-		byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-				key,
-				IntSerializer.INSTANCE,
-				VoidNamespace.INSTANCE,
-				VoidNamespaceSerializer.INSTANCE);
-
-		long requestId = Integer.MAX_VALUE + 182828L;
-
-		assertTrue(registryListener.registrationName.equals("vanilla"));
-
-		ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
-
-		// Write the request and wait for the response
-		channel.writeInbound(request);
-
-		Object msg = readInboundBlocking(channel);
-		assertTrue("Not ChunkedByteBuf", msg instanceof ChunkedByteBuf);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Queries the embedded channel for data.
-	 */
-	private Object readInboundBlocking(EmbeddedChannel channel) throws InterruptedException, TimeoutException {
-		final int sleepMillis = 50;
-
-		int sleptMillis = 0;
-
-		Object msg = null;
-		while (sleptMillis < READ_TIMEOUT_MILLIS &&
-				(msg = channel.readOutbound()) == null) {
-
-			Thread.sleep(sleepMillis);
-			sleptMillis += sleepMillis;
-		}
-
-		if (msg == null) {
-			throw new TimeoutException();
-		} else {
-			return msg;
-		}
-	}
-
-	/**
-	 * Frame length decoder (expected by the serialized messages).
-	 */
-	private ChannelHandler getFrameDecoder() {
-		return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4);
-	}
-
-	/**
-	 * A listener that keeps the last updated KvState information so that a test
-	 * can retrieve it.
-	 */
-	static class TestRegistryListener implements KvStateRegistryListener {
-		volatile JobVertexID jobVertexID;
-		volatile KeyGroupRange keyGroupIndex;
-		volatile String registrationName;
-		volatile KvStateID kvStateId;
-
-		@Override
-		public void notifyKvStateRegistered(JobID jobId,
-				JobVertexID jobVertexId,
-				KeyGroupRange keyGroupRange,
-				String registrationName,
-				KvStateID kvStateId) {
-			this.jobVertexID = jobVertexId;
-			this.keyGroupIndex = keyGroupRange;
-			this.registrationName = registrationName;
-			this.kvStateId = kvStateId;
-		}
-
-		@Override
-		public void notifyKvStateUnregistered(JobID jobId,
-				JobVertexID jobVertexId,
-				KeyGroupRange keyGroupRange,
-				String registrationName) {
-
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
deleted file mode 100644
index f8213e1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/netty/KvStateServerTest.java
+++ /dev/null
@@ -1,196 +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.query.netty;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestResult;
-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.KeyGroupRange;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import java.net.InetAddress;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for {@link KvStateServer}.
- */
-public class KvStateServerTest {
-
-	// Thread pool for client bootstrap (shared between tests)
-	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
-
-	private static final int TIMEOUT_MILLIS = 10000;
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (NIO_GROUP != null) {
-			NIO_GROUP.shutdownGracefully();
-		}
-	}
-
-	/**
-	 * Tests a simple successful query via a SocketChannel.
-	 */
-	@Test
-	public void testSimpleRequest() throws Exception {
-		KvStateServer server = null;
-		Bootstrap bootstrap = null;
-		try {
-			KvStateRegistry registry = new KvStateRegistry();
-			KvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-			server = new KvStateServer(InetAddress.getLocalHost(), 0, 1, 1, registry, stats);
-			server.start();
-
-			KvStateServerAddress serverAddress = server.getAddress();
-			int numKeyGroups = 1;
-			AbstractStateBackend abstractBackend = new MemoryStateBackend();
-			DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-			dummyEnv.setKvStateRegistry(registry);
-			AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-					dummyEnv,
-					new JobID(),
-					"test_op",
-					IntSerializer.INSTANCE,
-					numKeyGroups,
-					new KeyGroupRange(0, 0),
-					registry.createTaskRegistry(new JobID(), new JobVertexID()));
-
-			final KvStateServerHandlerTest.TestRegistryListener registryListener =
-					new KvStateServerHandlerTest.TestRegistryListener();
-
-			registry.registerListener(registryListener);
-
-			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-			desc.setQueryable("vanilla");
-
-			ValueState<Integer> state = backend.getPartitionedState(
-					VoidNamespace.INSTANCE,
-					VoidNamespaceSerializer.INSTANCE,
-					desc);
-
-			// Update KvState
-			int expectedValue = 712828289;
-
-			int key = 99812822;
-			backend.setCurrentKey(key);
-			state.update(expectedValue);
-
-			// Request
-			byte[] serializedKeyAndNamespace = KvStateRequestSerializer.serializeKeyAndNamespace(
-					key,
-					IntSerializer.INSTANCE,
-					VoidNamespace.INSTANCE,
-					VoidNamespaceSerializer.INSTANCE);
-
-			// Connect to the server
-			final BlockingQueue<ByteBuf> responses = new LinkedBlockingQueue<>();
-			bootstrap = createBootstrap(
-					new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4),
-					new ChannelInboundHandlerAdapter() {
-						@Override
-						public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-							responses.add((ByteBuf) msg);
-						}
-					});
-
-			Channel channel = bootstrap
-					.connect(serverAddress.getHost(), serverAddress.getPort())
-					.sync().channel();
-
-			long requestId = Integer.MAX_VALUE + 182828L;
-
-			assertTrue(registryListener.registrationName.equals("vanilla"));
-			ByteBuf request = KvStateRequestSerializer.serializeKvStateRequest(
-					channel.alloc(),
-					requestId,
-					registryListener.kvStateId,
-					serializedKeyAndNamespace);
-
-			channel.writeAndFlush(request);
-
-			ByteBuf buf = responses.poll(TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
-
-			assertEquals(KvStateRequestType.REQUEST_RESULT, KvStateRequestSerializer.deserializeHeader(buf));
-			KvStateRequestResult response = KvStateRequestSerializer.deserializeKvStateRequestResult(buf);
-
-			assertEquals(requestId, response.getRequestId());
-			int actualValue = KvStateRequestSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
-			assertEquals(expectedValue, actualValue);
-		} finally {
-			if (server != null) {
-				server.shutDown();
-			}
-
-			if (bootstrap != null) {
-				EventLoopGroup group = bootstrap.group();
-				if (group != null) {
-					group.shutdownGracefully();
-				}
-			}
-		}
-	}
-
-	/**
-	 * Creates a client bootstrap.
-	 */
-	private Bootstrap createBootstrap(final ChannelHandler... handlers) {
-		return new Bootstrap().group(NIO_GROUP).channel(NioSocketChannel.class)
-				.handler(new ChannelInitializer<SocketChannel>() {
-					@Override
-					protected void initChannel(SocketChannel ch) throws Exception {
-						ch.pipeline().addLast(handlers);
-					}
-				});
-	}
-
-}


[10/14] flink git commit: [FLINK-7770][QS] Hide the queryable state behind a proxy.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
deleted file mode 100644
index a2850b3..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
+++ /dev/null
@@ -1,752 +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.queryablestate.network;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.queryablestate.client.KvStateClient;
-import org.apache.flink.queryablestate.messages.KvStateRequest;
-import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.queryablestate.network.messages.MessageType;
-import org.apache.flink.queryablestate.server.KvStateServerImpl;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServer;
-import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-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.InternalKvState;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.util.NetUtils;
-
-import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
-import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
-import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link KvStateClient}.
- */
-public class KvStateClientTest {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientTest.class);
-
-	// Thread pool for client bootstrap (shared between tests)
-	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
-
-	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(100, TimeUnit.SECONDS);
-
-	@AfterClass
-	public static void tearDown() throws Exception {
-		if (NIO_GROUP != null) {
-			NIO_GROUP.shutdownGracefully();
-		}
-	}
-
-	/**
-	 * Tests simple queries, of which half succeed and half fail.
-	 */
-	@Test
-	public void testSimpleRequests() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			// Random result
-			final byte[] expected = new byte[1024];
-			ThreadLocalRandom.current().nextBytes(expected);
-
-			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.add((ByteBuf) msg);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			List<Future<byte[]>> futures = new ArrayList<>();
-
-			int numQueries = 1024;
-
-			for (int i = 0; i < numQueries; i++) {
-				futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-			}
-
-			// Respond to messages
-			Exception testException = new RuntimeException("Expected test Exception");
-
-			for (int i = 0; i < numQueries; i++) {
-				ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-				assertNotNull("Receive timed out", buf);
-
-				Channel ch = channel.get();
-				assertNotNull("Channel not active", ch);
-
-				assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
-				KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
-
-				buf.release();
-
-				if (i % 2 == 0) {
-					ByteBuf response = MessageSerializer.serializeKvStateRequestResult(
-							serverChannel.alloc(),
-							request.getRequestId(),
-							expected);
-
-					ch.writeAndFlush(response);
-				} else {
-					ByteBuf response = MessageSerializer.serializeKvStateRequestFailure(
-							serverChannel.alloc(),
-							request.getRequestId(),
-							testException);
-
-					ch.writeAndFlush(response);
-				}
-			}
-
-			for (int i = 0; i < numQueries; i++) {
-				if (i % 2 == 0) {
-					byte[] serializedResult = Await.result(futures.get(i), deadline.timeLeft());
-					assertArrayEquals(expected, serializedResult);
-				} else {
-					try {
-						Await.result(futures.get(i), deadline.timeLeft());
-						fail("Did not throw expected Exception");
-					} catch (RuntimeException ignored) {
-						// Expected
-					}
-				}
-			}
-
-			assertEquals(numQueries, stats.getNumRequests());
-			int expectedRequests = numQueries / 2;
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != expectedRequests ||
-					stats.getNumFailed() != expectedRequests)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(expectedRequests, stats.getNumSuccessful());
-			assertEquals(expectedRequests, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a request to an unavailable host is failed with ConnectException.
-	 */
-	@Test
-	public void testRequestUnavailableHost() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-		KvStateClient client = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			int availablePort = NetUtils.getAvailablePort();
-
-			KvStateServerAddress serverAddress = new KvStateServerAddress(
-					InetAddress.getLocalHost(),
-					availablePort);
-
-			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
-
-			try {
-				Await.result(future, deadline.timeLeft());
-				fail("Did not throw expected ConnectException");
-			} catch (ConnectException ignored) {
-				// Expected
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Multiple threads concurrently fire queries.
-	 */
-	@Test
-	public void testConcurrentQueries() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		ExecutorService executor = null;
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		final byte[] serializedResult = new byte[1024];
-		ThreadLocalRandom.current().nextBytes(serializedResult);
-
-		try {
-			int numQueryTasks = 4;
-			final int numQueriesPerTask = 1024;
-
-			executor = Executors.newFixedThreadPool(numQueryTasks);
-
-			client = new KvStateClient(1, stats);
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					ByteBuf buf = (ByteBuf) msg;
-					assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
-					KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
-
-					buf.release();
-
-					ByteBuf response = MessageSerializer.serializeKvStateRequestResult(
-							ctx.alloc(),
-							request.getRequestId(),
-							serializedResult);
-
-					ctx.channel().writeAndFlush(response);
-				}
-			});
-
-			final KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			final KvStateClient finalClient = client;
-			Callable<List<Future<byte[]>>> queryTask = new Callable<List<Future<byte[]>>>() {
-				@Override
-				public List<Future<byte[]>> call() throws Exception {
-					List<Future<byte[]>> results = new ArrayList<>(numQueriesPerTask);
-
-					for (int i = 0; i < numQueriesPerTask; i++) {
-						results.add(finalClient.getKvState(
-								serverAddress,
-								new KvStateID(),
-								new byte[0]));
-					}
-
-					return results;
-				}
-			};
-
-			// Submit query tasks
-			List<java.util.concurrent.Future<List<Future<byte[]>>>> futures = new ArrayList<>();
-			for (int i = 0; i < numQueryTasks; i++) {
-				futures.add(executor.submit(queryTask));
-			}
-
-			// Verify results
-			for (java.util.concurrent.Future<List<Future<byte[]>>> future : futures) {
-				List<Future<byte[]>> results = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-				for (Future<byte[]> result : results) {
-					byte[] actual = Await.result(result, deadline.timeLeft());
-					assertArrayEquals(serializedResult, actual);
-				}
-			}
-
-			int totalQueries = numQueryTasks * numQueriesPerTask;
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && stats.getNumSuccessful() != totalQueries) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(totalQueries, stats.getNumRequests());
-			assertEquals(totalQueries, stats.getNumSuccessful());
-		} finally {
-			if (executor != null) {
-				executor.shutdown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			if (client != null) {
-				client.shutDown();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a server failure closes the connection and removes it from
-	 * the established connections.
-	 */
-	@Test
-	public void testFailureClosesChannel() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.add((ByteBuf) msg);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			// Requests
-			List<Future<byte[]>> futures = new ArrayList<>();
-			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
-
-			ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			assertNotNull("Receive timed out", buf);
-			buf.release();
-
-			buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-			assertNotNull("Receive timed out", buf);
-			buf.release();
-
-			assertEquals(1, stats.getNumConnections());
-
-			Channel ch = channel.get();
-			assertNotNull("Channel not active", ch);
-
-			// Respond with failure
-			ch.writeAndFlush(MessageSerializer.serializeServerFailure(
-					serverChannel.alloc(),
-					new RuntimeException("Expected test server failure")));
-
-			try {
-				Await.result(futures.remove(0), deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (RuntimeException ignored) {
-				// Expected
-			}
-
-			try {
-				Await.result(futures.remove(0), deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (RuntimeException ignored) {
-				// Expected
-			}
-
-			assertEquals(0, stats.getNumConnections());
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
-					stats.getNumFailed() != 2)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(2, stats.getNumRequests());
-			assertEquals(0, stats.getNumSuccessful());
-			assertEquals(2, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests that a server channel close, closes the connection and removes it
-	 * from the established connections.
-	 */
-	@Test
-	public void testServerClosesChannel() throws Exception {
-		Deadline deadline = TEST_TIMEOUT.fromNow();
-		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		Channel serverChannel = null;
-
-		try {
-			client = new KvStateClient(1, stats);
-
-			final AtomicBoolean received = new AtomicBoolean();
-			final AtomicReference<Channel> channel = new AtomicReference<>();
-
-			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
-				@Override
-				public void channelActive(ChannelHandlerContext ctx) throws Exception {
-					channel.set(ctx.channel());
-				}
-
-				@Override
-				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-					received.set(true);
-				}
-			});
-
-			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
-
-			// Requests
-			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
-
-			while (!received.get() && deadline.hasTimeLeft()) {
-				Thread.sleep(50);
-			}
-			assertTrue("Receive timed out", received.get());
-
-			assertEquals(1, stats.getNumConnections());
-
-			channel.get().close().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
-
-			try {
-				Await.result(future, deadline.timeLeft());
-				fail("Did not throw expected server failure");
-			} catch (ClosedChannelException ignored) {
-				// Expected
-			}
-
-			assertEquals(0, stats.getNumConnections());
-
-			// Counts can take some time to propagate
-			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
-					stats.getNumFailed() != 1)) {
-				Thread.sleep(100);
-			}
-
-			assertEquals(1, stats.getNumRequests());
-			assertEquals(0, stats.getNumSuccessful());
-			assertEquals(1, stats.getNumFailed());
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			if (serverChannel != null) {
-				serverChannel.close();
-			}
-
-			assertEquals("Channel leak", 0, stats.getNumConnections());
-		}
-	}
-
-	/**
-	 * Tests multiple clients querying multiple servers until 100k queries have
-	 * been processed. At this point, the client is shut down and its verified
-	 * that all ongoing requests are failed.
-	 */
-	@Test
-	public void testClientServerIntegration() throws Exception {
-		// Config
-		final int numServers = 2;
-		final int numServerEventLoopThreads = 2;
-		final int numServerQueryThreads = 2;
-
-		final int numClientEventLoopThreads = 4;
-		final int numClientsTasks = 8;
-
-		final int batchSize = 16;
-
-		final int numKeyGroups = 1;
-
-		AbstractStateBackend abstractBackend = new MemoryStateBackend();
-		KvStateRegistry dummyRegistry = new KvStateRegistry();
-		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
-		dummyEnv.setKvStateRegistry(dummyRegistry);
-
-		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
-				dummyEnv,
-				new JobID(),
-				"test_op",
-				IntSerializer.INSTANCE,
-				numKeyGroups,
-				new KeyGroupRange(0, 0),
-				dummyRegistry.createTaskRegistry(new JobID(), new JobVertexID()));
-
-		final FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
-
-		AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats();
-
-		KvStateClient client = null;
-		ExecutorService clientTaskExecutor = null;
-		final KvStateServer[] server = new KvStateServer[numServers];
-
-		try {
-			client = new KvStateClient(numClientEventLoopThreads, clientStats);
-			clientTaskExecutor = Executors.newFixedThreadPool(numClientsTasks);
-
-			// Create state
-			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
-			desc.setQueryable("any");
-
-			// Create servers
-			KvStateRegistry[] registry = new KvStateRegistry[numServers];
-			AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
-			final KvStateID[] ids = new KvStateID[numServers];
-
-			for (int i = 0; i < numServers; i++) {
-				registry[i] = new KvStateRegistry();
-				serverStats[i] = new AtomicKvStateRequestStats();
-				server[i] = new KvStateServerImpl(
-						InetAddress.getLocalHost(),
-						0,
-						numServerEventLoopThreads,
-						numServerQueryThreads,
-						registry[i],
-						serverStats[i]);
-
-				server[i].start();
-
-				backend.setCurrentKey(1010 + i);
-
-				// Value per server
-				ValueState<Integer> state = backend.getPartitionedState(VoidNamespace.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE,
-						desc);
-
-				state.update(201 + i);
-
-				// we know it must be a KvStat but this is not exposed to the user via 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);
-			}
-
-			final KvStateClient finalClient = client;
-			Callable<Void> queryTask = new Callable<Void>() {
-				@Override
-				public Void call() throws Exception {
-					while (true) {
-						if (Thread.interrupted()) {
-							throw new InterruptedException();
-						}
-
-						// Random server permutation
-						List<Integer> random = new ArrayList<>();
-						for (int j = 0; j < batchSize; j++) {
-							random.add(j);
-						}
-						Collections.shuffle(random);
-
-						// Dispatch queries
-						List<Future<byte[]>> futures = new ArrayList<>(batchSize);
-
-						for (int j = 0; j < batchSize; j++) {
-							int targetServer = random.get(j) % numServers;
-
-							byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
-									1010 + targetServer,
-									IntSerializer.INSTANCE,
-									VoidNamespace.INSTANCE,
-									VoidNamespaceSerializer.INSTANCE);
-
-							futures.add(finalClient.getKvState(
-									server[targetServer].getAddress(),
-									ids[targetServer],
-									serializedKeyAndNamespace));
-						}
-
-						// Verify results
-						for (int j = 0; j < batchSize; j++) {
-							int targetServer = random.get(j) % numServers;
-
-							Future<byte[]> future = futures.get(j);
-							byte[] buf = Await.result(future, timeout);
-							int value = KvStateSerializer.deserializeValue(buf, IntSerializer.INSTANCE);
-							assertEquals(201 + targetServer, value);
-						}
-					}
-				}
-			};
-
-			// Submit tasks
-			List<java.util.concurrent.Future<Void>> taskFutures = new ArrayList<>();
-			for (int i = 0; i < numClientsTasks; i++) {
-				taskFutures.add(clientTaskExecutor.submit(queryTask));
-			}
-
-			long numRequests;
-			while ((numRequests = clientStats.getNumRequests()) < 100_000) {
-				Thread.sleep(100);
-				LOG.info("Number of requests {}/100_000", numRequests);
-			}
-
-			// Shut down
-			client.shutDown();
-
-			for (java.util.concurrent.Future<Void> future : taskFutures) {
-				try {
-					future.get();
-					fail("Did not throw expected Exception after shut down");
-				} catch (ExecutionException t) {
-					if (t.getCause() instanceof ClosedChannelException ||
-							t.getCause() instanceof IllegalStateException) {
-						// Expected
-					} else {
-						t.printStackTrace();
-						fail("Failed with unexpected Exception type: " + t.getClass().getName());
-					}
-				}
-			}
-
-			assertEquals("Connection leak (client)", 0, clientStats.getNumConnections());
-			for (int i = 0; i < numServers; i++) {
-				boolean success = false;
-				int numRetries = 0;
-				while (!success) {
-					try {
-						assertEquals("Connection leak (server)", 0, serverStats[i].getNumConnections());
-						success = true;
-					} catch (Throwable t) {
-						if (numRetries < 10) {
-							LOG.info("Retrying connection leak check (server)");
-							Thread.sleep((numRetries + 1) * 50);
-							numRetries++;
-						} else {
-							throw t;
-						}
-					}
-				}
-			}
-		} finally {
-			if (client != null) {
-				client.shutDown();
-			}
-
-			for (int i = 0; i < numServers; i++) {
-				if (server[i] != null) {
-					server[i].shutDown();
-				}
-			}
-
-			if (clientTaskExecutor != null) {
-				clientTaskExecutor.shutdown();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private Channel createServerChannel(final ChannelHandler... handlers) throws UnknownHostException, InterruptedException {
-		ServerBootstrap bootstrap = new ServerBootstrap()
-				// Bind address and port
-				.localAddress(InetAddress.getLocalHost(), 0)
-				// NIO server channels
-				.group(NIO_GROUP)
-				.channel(NioServerSocketChannel.class)
-				// See initializer for pipeline details
-				.childHandler(new ChannelInitializer<SocketChannel>() {
-					@Override
-					protected void initChannel(SocketChannel ch) throws Exception {
-						ch.pipeline()
-								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
-								.addLast(handlers);
-					}
-				});
-
-		return bootstrap.bind().sync().channel();
-	}
-
-	private KvStateServerAddress getKvStateServerAddress(Channel serverChannel) {
-		InetSocketAddress localAddress = (InetSocketAddress) serverChannel.localAddress();
-
-		return new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
deleted file mode 100644
index f28ca68..0000000
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
+++ /dev/null
@@ -1,214 +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.queryablestate.network;
-
-import org.apache.flink.queryablestate.messages.KvStateRequest;
-import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
-import org.apache.flink.queryablestate.messages.KvStateRequestResult;
-import org.apache.flink.queryablestate.network.messages.MessageSerializer;
-import org.apache.flink.queryablestate.network.messages.MessageType;
-import org.apache.flink.runtime.query.KvStateID;
-import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
-
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
-import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
-import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for {@link KvStateSerializer}.
- */
-@RunWith(Parameterized.class)
-public class KvStateRequestSerializerTest {
-
-	private final ByteBufAllocator alloc = UnpooledByteBufAllocator.DEFAULT;
-
-	@Parameterized.Parameters
-	public static Collection<Boolean> parameters() {
-		return Arrays.asList(false, true);
-	}
-
-	@Parameterized.Parameter
-	public boolean async;
-
-	/**
-	 * Tests KvState request serialization.
-	 */
-	@Test
-	public void testKvStateRequestSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 1337L;
-		KvStateID kvStateId = new KvStateID();
-		byte[] serializedKeyAndNamespace = randomByteArray(1024);
-
-		ByteBuf buf = MessageSerializer.serializeKvStateRequest(
-				alloc,
-				requestId,
-				kvStateId,
-				serializedKeyAndNamespace);
-
-		int frameLength = buf.readInt();
-		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
-		KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-		assertEquals(kvStateId, request.getKvStateId());
-		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
-	}
-
-	/**
-	 * Tests KvState request serialization with zero-length serialized key and namespace.
-	 */
-	@Test
-	public void testKvStateRequestSerializationWithZeroLengthKeyAndNamespace() throws Exception {
-		byte[] serializedKeyAndNamespace = new byte[0];
-
-		ByteBuf buf = MessageSerializer.serializeKvStateRequest(
-				alloc,
-				1823,
-				new KvStateID(),
-				serializedKeyAndNamespace);
-
-		int frameLength = buf.readInt();
-		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
-		KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
-	}
-
-	/**
-	 * Tests that we don't try to be smart about <code>null</code> key and namespace.
-	 * They should be treated explicitly.
-	 */
-	@Test(expected = NullPointerException.class)
-	public void testNullPointerExceptionOnNullSerializedKeyAndNamepsace() throws Exception {
-		new KvStateRequest(0, new KvStateID(), null);
-	}
-
-	/**
-	 * Tests KvState request result serialization.
-	 */
-	@Test
-	public void testKvStateRequestResultSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 72727278L;
-		byte[] serializedResult = randomByteArray(1024);
-
-		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
-				alloc,
-				requestId,
-				serializedResult);
-
-		int frameLength = buf.readInt();
-		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestResult request = MessageSerializer.deserializeKvStateRequestResult(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-
-		assertArrayEquals(serializedResult, request.getSerializedResult());
-	}
-
-	/**
-	 * Tests KvState request result serialization with zero-length serialized result.
-	 */
-	@Test
-	public void testKvStateRequestResultSerializationWithZeroLengthSerializedResult() throws Exception {
-		byte[] serializedResult = new byte[0];
-
-		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
-				alloc,
-				72727278,
-				serializedResult);
-
-		int frameLength = buf.readInt();
-
-		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestResult request = MessageSerializer.deserializeKvStateRequestResult(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertArrayEquals(serializedResult, request.getSerializedResult());
-	}
-
-	/**
-	 * Tests that we don't try to be smart about <code>null</code> results.
-	 * They should be treated explicitly.
-	 */
-	@Test(expected = NullPointerException.class)
-	public void testNullPointerExceptionOnNullSerializedResult() throws Exception {
-		new KvStateRequestResult(0, null);
-	}
-
-	/**
-	 * Tests KvState request failure serialization.
-	 */
-	@Test
-	public void testKvStateRequestFailureSerialization() throws Exception {
-		long requestId = Integer.MAX_VALUE + 1111222L;
-		IllegalStateException cause = new IllegalStateException("Expected test");
-
-		ByteBuf buf = MessageSerializer.serializeKvStateRequestFailure(
-				alloc,
-				requestId,
-				cause);
-
-		int frameLength = buf.readInt();
-		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure request = MessageSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(requestId, request.getRequestId());
-		assertEquals(cause.getClass(), request.getCause().getClass());
-		assertEquals(cause.getMessage(), request.getCause().getMessage());
-	}
-
-	/**
-	 * Tests KvState server failure serialization.
-	 */
-	@Test
-	public void testServerFailureSerialization() throws Exception {
-		IllegalStateException cause = new IllegalStateException("Expected test");
-
-		ByteBuf buf = MessageSerializer.serializeServerFailure(alloc, cause);
-
-		int frameLength = buf.readInt();
-		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
-		Throwable request = MessageSerializer.deserializeServerFailure(buf);
-		assertEquals(buf.readerIndex(), frameLength + 4);
-
-		assertEquals(cause.getClass(), request.getClass());
-		assertEquals(cause.getMessage(), request.getMessage());
-	}
-
-	private byte[] randomByteArray(int capacity) {
-		byte[] bytes = new byte[capacity];
-		ThreadLocalRandom.current().nextBytes(bytes);
-		return bytes;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
index c37c822..944349ee 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java
@@ -24,20 +24,22 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
-import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
-import org.apache.flink.queryablestate.UnknownKvStateID;
-import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
-import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespaceException;
+import org.apache.flink.queryablestate.UnknownKvStateIdException;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
 import org.apache.flink.queryablestate.network.messages.MessageSerializer;
 import org.apache.flink.queryablestate.network.messages.MessageType;
-import org.apache.flink.queryablestate.server.ChunkedByteBuf;
+import org.apache.flink.queryablestate.network.messages.RequestFailure;
 import org.apache.flink.queryablestate.server.KvStateServerHandler;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
 import org.apache.flink.runtime.query.KvStateID;
 import org.apache.flink.runtime.query.KvStateRegistry;
 import org.apache.flink.runtime.query.KvStateRegistryListener;
 import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.KvStateRequestStats;
 import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
@@ -57,10 +59,11 @@ import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -76,16 +79,28 @@ import static org.mockito.Mockito.when;
  */
 public class KvStateServerHandlerTest extends TestLogger {
 
-	/** Shared Thread pool for query execution. */
-	private static final ExecutorService TEST_THREAD_POOL = Executors.newSingleThreadExecutor();
-
-	private static final int READ_TIMEOUT_MILLIS = 10000;
+	private static KvStateServerImpl testServer;
+
+	private static final long READ_TIMEOUT_MILLIS = 10000L;
+
+	@BeforeClass
+	public static void setup() {
+		try {
+			testServer = new KvStateServerImpl(
+					InetAddress.getLocalHost(),
+					0,
+					1,
+					1,
+					new KvStateRegistry(),
+					new DisabledKvStateRequestStats());
+		} catch (UnknownHostException e) {
+			e.printStackTrace();
+		}
+	}
 
 	@AfterClass
 	public static void tearDown() throws Exception {
-		if (TEST_THREAD_POOL != null) {
-			TEST_THREAD_POOL.shutdown();
-		}
+		testServer.shutdown();
 	}
 
 	/**
@@ -96,7 +111,10 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		// Register state
@@ -141,40 +159,40 @@ public class KvStateServerHandlerTest extends TestLogger {
 
 		assertTrue(registryListener.registrationName.equals("vanilla"));
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
+		KvStateInternalRequest request = new KvStateInternalRequest(
+				registryListener.kvStateId, serializedKeyAndNamespace);
+
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), requestId, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestResult response = MessageSerializer.deserializeKvStateRequestResult(buf);
+		long deserRequestId = MessageSerializer.getRequestId(buf);
+		KvStateResponse response = serializer.deserializeResponse(buf);
 
-		assertEquals(requestId, response.getRequestId());
+		assertEquals(requestId, deserRequestId);
 
-		int actualValue = KvStateSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
+		int actualValue = KvStateSerializer.deserializeValue(response.getContent(), IntSerializer.INSTANCE);
 		assertEquals(expectedValue, actualValue);
 
 		assertEquals(stats.toString(), 1, stats.getNumRequests());
 
 		// Wait for async successful request report
 		long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
-		while (stats.getNumSuccessful() != 1 && System.nanoTime() <= deadline) {
-			Thread.sleep(10);
+		while (stats.getNumSuccessful() != 1L && System.nanoTime() <= deadline) {
+			Thread.sleep(10L);
 		}
 
-		assertEquals(stats.toString(), 1, stats.getNumSuccessful());
+		assertEquals(stats.toString(), 1L, stats.getNumSuccessful());
 	}
 
 	/**
-	 * Tests the failure response with {@link UnknownKvStateID} as cause on
+	 * Tests the failure response with {@link UnknownKvStateIdException} as cause on
 	 * queries for unregistered KvStateIDs.
 	 */
 	@Test
@@ -182,36 +200,38 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		long requestId = Integer.MAX_VALUE + 182828L;
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				new KvStateID(),
-				new byte[0]);
+
+		KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), requestId, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
 
 		assertEquals(requestId, response.getRequestId());
 
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateID);
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKvStateIdException);
 
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
 	}
 
 	/**
-	 * Tests the failure response with {@link UnknownKeyOrNamespace} as cause
+	 * Tests the failure response with {@link UnknownKeyOrNamespaceException} as cause
 	 * on queries for non-existing keys.
 	 */
 	@Test
@@ -219,7 +239,10 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		int numKeyGroups = 1;
@@ -254,40 +277,39 @@ public class KvStateServerHandlerTest extends TestLogger {
 
 		assertTrue(registryListener.registrationName.equals("vanilla"));
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
+		KvStateInternalRequest request = new KvStateInternalRequest(registryListener.kvStateId, serializedKeyAndNamespace);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), requestId, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
 
 		assertEquals(requestId, response.getRequestId());
 
-		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespace);
+		assertTrue("Did not respond with expected failure cause", response.getCause() instanceof UnknownKeyOrNamespaceException);
 
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
 	}
 
 	/**
-	 * Tests the failure response on a failure on the {@link InternalKvState#getSerializedValue(byte[])}
-	 * call.
+	 * Tests the failure response on a failure on the {@link InternalKvState#getSerializedValue(byte[])} call.
 	 */
 	@Test
 	public void testFailureOnGetSerializedValue() throws Exception {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		// Failing KvState
@@ -302,38 +324,37 @@ public class KvStateServerHandlerTest extends TestLogger {
 				"vanilla",
 				kvState);
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				kvStateId,
-				new byte[0]);
+		KvStateInternalRequest request = new KvStateInternalRequest(kvStateId, new byte[0]);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 282872L, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
 
 		assertTrue(response.getCause().getMessage().contains("Expected test Exception"));
 
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
 	}
 
 	/**
-	 * Tests that the channel is closed if an Exception reaches the channel
-	 * handler.
+	 * Tests that the channel is closed if an Exception reaches the channel handler.
 	 */
 	@Test
 	public void testCloseChannelOnExceptionCaught() throws Exception {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(handler);
 
 		channel.pipeline().fireExceptionCaught(new RuntimeException("Expected test Exception"));
@@ -352,19 +373,28 @@ public class KvStateServerHandlerTest extends TestLogger {
 	}
 
 	/**
-	 * Tests the failure response on a rejected execution, because the query
-	 * executor has been closed.
+	 * Tests the failure response on a rejected execution, because the query executor has been closed.
 	 */
 	@Test
 	public void testQueryExecutorShutDown() throws Exception {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		ExecutorService closedExecutor = Executors.newSingleThreadExecutor();
-		closedExecutor.shutdown();
-		assertTrue(closedExecutor.isShutdown());
+		KvStateServerImpl localTestServer = new KvStateServerImpl(
+				InetAddress.getLocalHost(),
+				0,
+				1,
+				1,
+				new KvStateRegistry(),
+				new DisabledKvStateRequestStats());
+
+		localTestServer.shutdown();
+		assertTrue(localTestServer.isExecutorShutdown());
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, closedExecutor, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(localTestServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		int numKeyGroups = 1;
@@ -391,26 +421,25 @@ public class KvStateServerHandlerTest extends TestLogger {
 
 		assertTrue(registryListener.registrationName.equals("vanilla"));
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				registryListener.kvStateId,
-				new byte[0]);
+		KvStateInternalRequest request = new KvStateInternalRequest(registryListener.kvStateId, new byte[0]);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 282872L, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
 
 		assertTrue(response.getCause().getMessage().contains("RejectedExecutionException"));
 
-		assertEquals(1, stats.getNumRequests());
-		assertEquals(1, stats.getNumFailed());
+		assertEquals(1L, stats.getNumRequests());
+		assertEquals(1L, stats.getNumFailed());
+
+		localTestServer.shutdown();
 	}
 
 	/**
@@ -421,7 +450,10 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		// Write the request and wait for the response
@@ -438,13 +470,11 @@ public class KvStateServerHandlerTest extends TestLogger {
 		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
 		Throwable response = MessageSerializer.deserializeServerFailure(buf);
 
-		assertEquals(0, stats.getNumRequests());
-		assertEquals(0, stats.getNumFailed());
+		assertEquals(0L, stats.getNumRequests());
+		assertEquals(0L, stats.getNumFailed());
 
-		unexpectedMessage = MessageSerializer.serializeKvStateRequestResult(
-				channel.alloc(),
-				192,
-				new byte[0]);
+		KvStateResponse stateResponse = new KvStateResponse(new byte[0]);
+		unexpectedMessage = MessageSerializer.serializeResponse(channel.alloc(), 192L, stateResponse);
 
 		channel.writeInbound(unexpectedMessage);
 
@@ -457,8 +487,8 @@ public class KvStateServerHandlerTest extends TestLogger {
 
 		assertTrue("Unexpected failure cause " + response.getClass().getName(), response instanceof IllegalArgumentException);
 
-		assertEquals(0, stats.getNumRequests());
-		assertEquals(0, stats.getNumFailed());
+		assertEquals(0L, stats.getNumRequests());
+		assertEquals(0L, stats.getNumFailed());
 	}
 
 	/**
@@ -469,30 +499,30 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				282872,
-				new KvStateID(),
-				new byte[0]);
+		KvStateInternalRequest request = new KvStateInternalRequest(new KvStateID(), new byte[0]);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 282872L, request);
 
-		assertEquals(1, request.refCnt());
+		assertEquals(1L, serRequest.refCnt());
 
 		// Write regular request
-		channel.writeInbound(request);
-		assertEquals("Buffer not recycled", 0, request.refCnt());
+		channel.writeInbound(serRequest);
+		assertEquals("Buffer not recycled", 0L, serRequest.refCnt());
 
 		// Write unexpected msg
 		ByteBuf unexpected = channel.alloc().buffer(8);
 		unexpected.writeInt(4);
 		unexpected.writeInt(4);
 
-		assertEquals(1, unexpected.refCnt());
+		assertEquals(1L, unexpected.refCnt());
 
 		channel.writeInbound(unexpected);
-		assertEquals("Buffer not recycled", 0, unexpected.refCnt());
+		assertEquals("Buffer not recycled", 0L, unexpected.refCnt());
 	}
 
 	/**
@@ -503,7 +533,10 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		int numKeyGroups = 1;
@@ -550,45 +583,40 @@ public class KvStateServerHandlerTest extends TestLogger {
 				StringSerializer.INSTANCE);
 
 		assertTrue(registryListener.registrationName.equals("vanilla"));
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				182828,
-				registryListener.kvStateId,
-				wrongKeyAndNamespace);
+
+		KvStateInternalRequest request = new KvStateInternalRequest(registryListener.kvStateId, wrongKeyAndNamespace);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), 182828L, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		ByteBuf buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		KvStateRequestFailure response = MessageSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(182828, response.getRequestId());
+		RequestFailure response = MessageSerializer.deserializeRequestFailure(buf);
+		assertEquals(182828L, response.getRequestId());
 		assertTrue(response.getCause().getMessage().contains("IOException"));
 
 		// Repeat with wrong namespace only
-		request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				182829,
-				registryListener.kvStateId,
-				wrongNamespace);
+		request = new KvStateInternalRequest(registryListener.kvStateId, wrongNamespace);
+		serRequest = MessageSerializer.serializeRequest(channel.alloc(), 182829L, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		buf = (ByteBuf) readInboundBlocking(channel);
 		buf.skipBytes(4); // skip frame length
 
 		// Verify the response
 		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
-		response = MessageSerializer.deserializeKvStateRequestFailure(buf);
-		assertEquals(182829, response.getRequestId());
+		response = MessageSerializer.deserializeRequestFailure(buf);
+		assertEquals(182829L, response.getRequestId());
 		assertTrue(response.getCause().getMessage().contains("IOException"));
 
-		assertEquals(2, stats.getNumRequests());
-		assertEquals(2, stats.getNumFailed());
+		assertEquals(2L, stats.getNumRequests());
+		assertEquals(2L, stats.getNumFailed());
 	}
 
 	/**
@@ -599,7 +627,10 @@ public class KvStateServerHandlerTest extends TestLogger {
 		KvStateRegistry registry = new KvStateRegistry();
 		KvStateRequestStats stats = new AtomicKvStateRequestStats();
 
-		KvStateServerHandler handler = new KvStateServerHandler(registry, TEST_THREAD_POOL, stats);
+		MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		KvStateServerHandler handler = new KvStateServerHandler(testServer, registry, serializer, stats);
 		EmbeddedChannel channel = new EmbeddedChannel(getFrameDecoder(), handler);
 
 		int numKeyGroups = 1;
@@ -650,14 +681,11 @@ public class KvStateServerHandlerTest extends TestLogger {
 
 		assertTrue(registryListener.registrationName.equals("vanilla"));
 
-		ByteBuf request = MessageSerializer.serializeKvStateRequest(
-				channel.alloc(),
-				requestId,
-				registryListener.kvStateId,
-				serializedKeyAndNamespace);
+		KvStateInternalRequest request = new KvStateInternalRequest(registryListener.kvStateId, serializedKeyAndNamespace);
+		ByteBuf serRequest = MessageSerializer.serializeRequest(channel.alloc(), requestId, request);
 
 		// Write the request and wait for the response
-		channel.writeInbound(request);
+		channel.writeInbound(serRequest);
 
 		Object msg = readInboundBlocking(channel);
 		assertTrue("Not ChunkedByteBuf", msg instanceof ChunkedByteBuf);
@@ -669,9 +697,9 @@ public class KvStateServerHandlerTest extends TestLogger {
 	 * Queries the embedded channel for data.
 	 */
 	private Object readInboundBlocking(EmbeddedChannel channel) throws InterruptedException, TimeoutException {
-		final int sleepMillis = 50;
+		final long sleepMillis = 50L;
 
-		int sleptMillis = 0;
+		long sleptMillis = 0L;
 
 		Object msg = null;
 		while (sleptMillis < READ_TIMEOUT_MILLIS &&

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
index 9332e68..b7f489a 100644
--- a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
@@ -22,14 +22,14 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
 import org.apache.flink.queryablestate.network.messages.MessageSerializer;
 import org.apache.flink.queryablestate.network.messages.MessageType;
 import org.apache.flink.queryablestate.server.KvStateServerImpl;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
 import org.apache.flink.runtime.query.KvStateRegistry;
-import org.apache.flink.runtime.query.KvStateServer;
 import org.apache.flink.runtime.query.KvStateServerAddress;
 import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
 import org.apache.flink.runtime.query.netty.KvStateRequestStats;
@@ -66,7 +66,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Tests for {@link KvStateServer}.
+ * Tests for {@link KvStateServerImpl}.
  */
 public class KvStateServerTest {
 
@@ -87,7 +87,7 @@ public class KvStateServerTest {
 	 */
 	@Test
 	public void testSimpleRequest() throws Exception {
-		KvStateServer server = null;
+		KvStateServerImpl server = null;
 		Bootstrap bootstrap = null;
 		try {
 			KvStateRegistry registry = new KvStateRegistry();
@@ -96,7 +96,7 @@ public class KvStateServerTest {
 			server = new KvStateServerImpl(InetAddress.getLocalHost(), 0, 1, 1, registry, stats);
 			server.start();
 
-			KvStateServerAddress serverAddress = server.getAddress();
+			KvStateServerAddress serverAddress = server.getServerAddress();
 			int numKeyGroups = 1;
 			AbstractStateBackend abstractBackend = new MemoryStateBackend();
 			DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
@@ -155,25 +155,29 @@ public class KvStateServerTest {
 			long requestId = Integer.MAX_VALUE + 182828L;
 
 			assertTrue(registryListener.registrationName.equals("vanilla"));
-			ByteBuf request = MessageSerializer.serializeKvStateRequest(
-					channel.alloc(),
-					requestId,
+
+			final KvStateInternalRequest request = new KvStateInternalRequest(
 					registryListener.kvStateId,
 					serializedKeyAndNamespace);
 
-			channel.writeAndFlush(request);
+			ByteBuf serializeRequest = MessageSerializer.serializeRequest(
+					channel.alloc(),
+					requestId,
+					request);
+
+			channel.writeAndFlush(serializeRequest);
 
 			ByteBuf buf = responses.poll(TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
 
 			assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
-			KvStateRequestResult response = MessageSerializer.deserializeKvStateRequestResult(buf);
+			assertEquals(requestId, MessageSerializer.getRequestId(buf));
+			KvStateResponse response = server.getSerializer().deserializeResponse(buf);
 
-			assertEquals(requestId, response.getRequestId());
-			int actualValue = KvStateSerializer.deserializeValue(response.getSerializedResult(), IntSerializer.INSTANCE);
+			int actualValue = KvStateSerializer.deserializeValue(response.getContent(), IntSerializer.INSTANCE);
 			assertEquals(expectedValue, actualValue);
 		} finally {
 			if (server != null) {
-				server.shutDown();
+				server.shutdown();
 			}
 
 			if (bootstrap != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f48f5340/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
new file mode 100644
index 0000000..32a0c9b
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/MessageSerializerTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.queryablestate.messages.KvStateInternalRequest;
+import org.apache.flink.queryablestate.messages.KvStateResponse;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.network.messages.RequestFailure;
+import org.apache.flink.runtime.query.KvStateID;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link MessageSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class MessageSerializerTest {
+
+	private final ByteBufAllocator alloc = UnpooledByteBufAllocator.DEFAULT;
+
+	@Parameterized.Parameters
+	public static Collection<Boolean> parameters() {
+		return Arrays.asList(false, true);
+	}
+
+	@Parameterized.Parameter
+	public boolean async;
+
+	/**
+	 * Tests request serialization.
+	 */
+	@Test
+	public void testRequestSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 1337L;
+		KvStateID kvStateId = new KvStateID();
+		byte[] serializedKeyAndNamespace = randomByteArray(1024);
+
+		final KvStateInternalRequest request = new KvStateInternalRequest(kvStateId, serializedKeyAndNamespace);
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		ByteBuf buf = MessageSerializer.serializeRequest(alloc, requestId, request);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+		assertEquals(requestId, MessageSerializer.getRequestId(buf));
+		KvStateInternalRequest requestDeser = serializer.deserializeRequest(buf);
+
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(kvStateId, requestDeser.getKvStateId());
+		assertArrayEquals(serializedKeyAndNamespace, requestDeser.getSerializedKeyAndNamespace());
+	}
+
+	/**
+	 * Tests request serialization with zero-length serialized key and namespace.
+	 */
+	@Test
+	public void testRequestSerializationWithZeroLengthKeyAndNamespace() throws Exception {
+
+		long requestId = Integer.MAX_VALUE + 1337L;
+		KvStateID kvStateId = new KvStateID();
+		byte[] serializedKeyAndNamespace = new byte[0];
+
+		final KvStateInternalRequest request = new KvStateInternalRequest(kvStateId, serializedKeyAndNamespace);
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		ByteBuf buf = MessageSerializer.serializeRequest(alloc, requestId, request);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+		assertEquals(requestId, MessageSerializer.getRequestId(buf));
+		KvStateInternalRequest requestDeser = serializer.deserializeRequest(buf);
+
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(kvStateId, requestDeser.getKvStateId());
+		assertArrayEquals(serializedKeyAndNamespace, requestDeser.getSerializedKeyAndNamespace());
+	}
+
+	/**
+	 * Tests that we don't try to be smart about <code>null</code> key and namespace.
+	 * They should be treated explicitly.
+	 */
+	@Test(expected = NullPointerException.class)
+	public void testNullPointerExceptionOnNullSerializedKeyAndNamepsace() throws Exception {
+		new KvStateInternalRequest(new KvStateID(), null);
+	}
+
+	/**
+	 * Tests response serialization.
+	 */
+	@Test
+	public void testResponseSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 72727278L;
+		byte[] serializedResult = randomByteArray(1024);
+
+		final KvStateResponse response = new KvStateResponse(serializedResult);
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		ByteBuf buf = MessageSerializer.serializeResponse(alloc, requestId, response);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		assertEquals(requestId, MessageSerializer.getRequestId(buf));
+		KvStateResponse responseDeser = serializer.deserializeResponse(buf);
+
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertArrayEquals(serializedResult, responseDeser.getContent());
+	}
+
+	/**
+	 * Tests response serialization with zero-length serialized result.
+	 */
+	@Test
+	public void testResponseSerializationWithZeroLengthSerializedResult() throws Exception {
+		byte[] serializedResult = new byte[0];
+
+		final KvStateResponse response = new KvStateResponse(serializedResult);
+		final MessageSerializer<KvStateInternalRequest, KvStateResponse> serializer =
+				new MessageSerializer<>(new KvStateInternalRequest.KvStateInternalRequestDeserializer(), new KvStateResponse.KvStateResponseDeserializer());
+
+		ByteBuf buf = MessageSerializer.serializeResponse(alloc, 72727278L, response);
+
+		int frameLength = buf.readInt();
+
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		assertEquals(72727278L, MessageSerializer.getRequestId(buf));
+		KvStateResponse responseDeser = serializer.deserializeResponse(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertArrayEquals(serializedResult, responseDeser.getContent());
+	}
+
+	/**
+	 * Tests that we don't try to be smart about <code>null</code> results.
+	 * They should be treated explicitly.
+	 */
+	@Test(expected = NullPointerException.class)
+	public void testNullPointerExceptionOnNullSerializedResult() throws Exception {
+		new KvStateResponse((byte[]) null);
+	}
+
+	/**
+	 * Tests request failure serialization.
+	 */
+	@Test
+	public void testKvStateRequestFailureSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 1111222L;
+		IllegalStateException cause = new IllegalStateException("Expected test");
+
+		ByteBuf buf = MessageSerializer.serializeRequestFailure(alloc, requestId, cause);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		RequestFailure requestFailure = MessageSerializer.deserializeRequestFailure(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(requestId, requestFailure.getRequestId());
+		assertEquals(cause.getClass(), requestFailure.getCause().getClass());
+		assertEquals(cause.getMessage(), requestFailure.getCause().getMessage());
+	}
+
+	/**
+	 * Tests server failure serialization.
+	 */
+	@Test
+	public void testServerFailureSerialization() throws Exception {
+		IllegalStateException cause = new IllegalStateException("Expected test");
+
+		ByteBuf buf = MessageSerializer.serializeServerFailure(alloc, cause);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
+		Throwable request = MessageSerializer.deserializeServerFailure(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(cause.getClass(), request.getClass());
+		assertEquals(cause.getMessage(), request.getMessage());
+	}
+
+	private byte[] randomByteArray(int capacity) {
+		byte[] bytes = new byte[capacity];
+		ThreadLocalRandom.current().nextBytes(bytes);
+		return bytes;
+	}
+}


[08/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
[FLINK-7769][QS] Move queryable state outside the runtime.

Creates a separate for the queryable state and  moves the client
code outside the runtime. The Task Manager is now instantiating
the KvStateServer using reflection.


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

Branch: refs/heads/master
Commit: 29a6e9952ebb2c7349d25d3696e2ec1d7e8e620a
Parents: bc4638a
Author: kkloudas <kk...@gmail.com>
Authored: Wed Oct 4 19:11:09 2017 +0200
Committer: kkloudas <kk...@gmail.com>
Committed: Wed Oct 11 15:33:30 2017 +0200

----------------------------------------------------------------------
 .../streaming/state/AbstractRocksDBState.java   |    6 +-
 .../streaming/state/RocksDBMapState.java        |    8 +-
 .../flink-queryable-state-java/pom.xml          |  143 +++
 .../flink/queryablestate/UnknownJobManager.java |   35 +
 .../queryablestate/UnknownKeyOrNamespace.java   |   31 +
 .../flink/queryablestate/UnknownKvStateID.java  |   35 +
 .../UnknownKvStateKeyGroupLocation.java         |   31 +
 .../AkkaKvStateLocationLookupService.java       |  325 +++++
 .../queryablestate/client/KvStateClient.java    |  583 +++++++++
 .../client/KvStateClientHandler.java            |  107 ++
 .../client/KvStateClientHandlerCallback.java    |   54 +
 .../client/KvStateLocationLookupService.java    |   51 +
 .../client/QueryableStateClient.java            |  590 +++++++++
 .../queryablestate/messages/KvStateRequest.java |   89 ++
 .../messages/KvStateRequestFailure.java         |   68 ++
 .../messages/KvStateRequestResult.java          |   74 ++
 .../network/messages/MessageSerializer.java     |  332 ++++++
 .../network/messages/MessageType.java           |   39 +
 .../queryablestate/server/ChunkedByteBuf.java   |   98 ++
 .../server/KvStateServerHandler.java            |  308 +++++
 .../server/KvStateServerImpl.java               |  230 ++++
 .../itcases/AbstractQueryableStateITCase.java   | 1128 ++++++++++++++++++
 .../itcases/HAAbstractQueryableStateITCase.java |  101 ++
 .../HAQueryableStateITCaseFsBackend.java        |   39 +
 .../HAQueryableStateITCaseRocksDBBackend.java   |   39 +
 .../KVStateRequestSerializerRocksDBTest.java    |  167 +++
 .../NonHAAbstractQueryableStateITCase.java      |   81 ++
 .../NonHAQueryableStateITCaseFsBackend.java     |   39 +
 ...NonHAQueryableStateITCaseRocksDBBackend.java |   39 +
 .../AkkaKvStateLocationLookupServiceTest.java   |  399 +++++++
 .../network/KvStateClientHandlerTest.java       |  117 ++
 .../network/KvStateClientTest.java              |  752 ++++++++++++
 .../network/KvStateRequestSerializerTest.java   |  214 ++++
 .../network/KvStateServerHandlerTest.java       |  728 +++++++++++
 .../network/KvStateServerTest.java              |  201 ++++
 .../network/QueryableStateClientTest.java       |  458 +++++++
 .../src/test/resources/log4j-test.properties    |   31 +
 flink-queryable-state/pom.xml                   |   54 +
 .../runtime/io/network/NetworkEnvironment.java  |    2 +-
 .../query/AkkaKvStateLocationLookupService.java |  322 -----
 .../flink/runtime/query/KvStateLocation.java    |   10 +-
 .../query/KvStateLocationLookupService.java     |   50 -
 .../flink/runtime/query/KvStateRegistry.java    |    1 -
 .../flink/runtime/query/KvStateServer.java      |   43 +
 .../runtime/query/KvStateServerAddress.java     |    6 +-
 .../runtime/query/QueryableStateClient.java     |  587 ---------
 .../runtime/query/QueryableStateUtils.java      |   89 ++
 .../flink/runtime/query/UnknownJobManager.java  |   33 -
 .../query/UnknownKvStateKeyGroupLocation.java   |   29 -
 .../runtime/query/netty/ChunkedByteBuf.java     |   98 --
 .../runtime/query/netty/KvStateClient.java      |  579 ---------
 .../query/netty/KvStateClientHandler.java       |  106 --
 .../netty/KvStateClientHandlerCallback.java     |   54 -
 .../query/netty/KvStateRequestStats.java        |    2 +
 .../runtime/query/netty/KvStateServer.java      |  239 ----
 .../query/netty/KvStateServerHandler.java       |  305 -----
 .../query/netty/UnknownKeyOrNamespace.java      |   31 -
 .../runtime/query/netty/UnknownKvStateID.java   |   35 -
 .../query/netty/message/KvStateRequest.java     |   89 --
 .../netty/message/KvStateRequestFailure.java    |   68 --
 .../netty/message/KvStateRequestResult.java     |   74 --
 .../netty/message/KvStateRequestSerializer.java |  568 ---------
 .../query/netty/message/KvStateRequestType.java |   40 -
 .../query/netty/message/KvStateSerializer.java  |  267 +++++
 .../flink/runtime/query/netty/package-info.java |   80 --
 .../flink/runtime/query/package-info.java       |   60 -
 .../runtime/state/heap/AbstractHeapState.java   |    6 +-
 .../flink/runtime/state/heap/HeapMapState.java  |    4 +-
 .../QueryableStateConfiguration.java            |    6 +-
 .../taskexecutor/TaskManagerServices.java       |   34 +-
 .../TaskManagerServicesConfiguration.java       |    9 +-
 .../AkkaKvStateLocationLookupServiceTest.java   |  398 ------
 .../runtime/query/QueryableStateClientTest.java |  449 -------
 .../query/netty/KvStateClientHandlerTest.java   |  115 --
 .../runtime/query/netty/KvStateClientTest.java  |  747 ------------
 .../query/netty/KvStateServerHandlerTest.java   |  721 -----------
 .../runtime/query/netty/KvStateServerTest.java  |  196 ---
 .../message/KvStateRequestSerializerTest.java   |  219 +---
 .../runtime/state/StateBackendTestBase.java     |   14 +-
 .../query/AbstractQueryableStateITCase.java     | 1128 ------------------
 .../query/HAAbstractQueryableStateITCase.java   |  102 --
 .../query/HAQueryableStateITCaseFsBackend.java  |   39 -
 .../HAQueryableStateITCaseRocksDBBackend.java   |   39 -
 .../KVStateRequestSerializerRocksDBTest.java    |  168 ---
 .../NonHAAbstractQueryableStateITCase.java      |   81 --
 .../NonHAQueryableStateITCaseFsBackend.java     |   39 -
 ...NonHAQueryableStateITCaseRocksDBBackend.java |   39 -
 pom.xml                                         |    1 +
 88 files changed, 8299 insertions(+), 7946 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/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 c061835..cf365b4 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
@@ -27,7 +27,7 @@ import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.util.Preconditions;
@@ -125,8 +125,8 @@ public abstract class AbstractRocksDBState<K, N, S extends State, SD extends Sta
 	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
 		Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
 
-		//TODO make KvStateRequestSerializer key-group aware to save this round trip and key-group computation
-		Tuple2<K, N> des = KvStateRequestSerializer.<K, N>deserializeKeyAndNamespace(
+		//TODO make KvStateSerializer key-group aware to save this round trip and key-group computation
+		Tuple2<K, N> des = KvStateSerializer.<K, N>deserializeKeyAndNamespace(
 				serializedKeyAndNamespace,
 				backend.getKeySerializer(),
 				namespaceSerializer);

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
index 75c1651..421bb2e 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
@@ -26,7 +26,7 @@ import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
 import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
 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.query.netty.message.KvStateSerializer;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.util.Preconditions;
@@ -223,8 +223,8 @@ public class RocksDBMapState<K, N, UK, UV>
 	public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) throws Exception {
 		Preconditions.checkNotNull(serializedKeyAndNamespace, "Serialized key and namespace");
 
-		//TODO make KvStateRequestSerializer key-group aware to save this round trip and key-group computation
-		Tuple2<K, N> des = KvStateRequestSerializer.deserializeKeyAndNamespace(
+		//TODO make KvStateSerializer key-group aware to save this round trip and key-group computation
+		Tuple2<K, N> des = KvStateSerializer.deserializeKeyAndNamespace(
 				serializedKeyAndNamespace,
 				backend.getKeySerializer(),
 				namespaceSerializer);
@@ -248,7 +248,7 @@ public class RocksDBMapState<K, N, UK, UV>
 			return null;
 		}
 
-		return KvStateRequestSerializer.serializeMap(new Iterable<Map.Entry<UK, UV>>() {
+		return KvStateSerializer.serializeMap(new Iterable<Map.Entry<UK, UV>>() {
 			@Override
 			public Iterator<Map.Entry<UK, UV>> iterator() {
 				return iterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/pom.xml b/flink-queryable-state/flink-queryable-state-java/pom.xml
new file mode 100644
index 0000000..63403df
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/pom.xml
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-queryable-state</artifactId>
+        <version>1.4-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+
+    <artifactId>flink-queryable-state-java_${scala.binary.version}</artifactId>
+    <name>flink-queryable-state-java</name>
+    <packaging>jar</packaging>
+
+    <dependencies>
+
+        <!-- core dependencies -->
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-core</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <!-- test dependencies -->
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+       <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- ===================================================
+								Testing
+			=================================================== -->
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils-junit</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <version>${curator.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>com.data-artisans</groupId>
+            <artifactId>flakka-testkit_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-runtime_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+    
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
new file mode 100644
index 0000000..93f2ba5
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownJobManager.java
@@ -0,0 +1,35 @@
+/*
+ * 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.queryablestate;
+
+import org.apache.flink.queryablestate.client.KvStateLocationLookupService;
+
+/**
+ * Exception to fail Future with if no JobManager is currently registered at
+ * the {@link KvStateLocationLookupService}.
+ */
+public class UnknownJobManager extends Exception {
+
+	private static final long serialVersionUID = 1L;
+
+	public UnknownJobManager() {
+		super("Unknown JobManager. Either the JobManager has not registered yet " +
+				"or has lost leadership.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.java
new file mode 100644
index 0000000..e921e40
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKeyOrNamespace.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.queryablestate;
+
+/**
+ * Thrown if the KvState does not hold any state for the given key or namespace.
+ */
+public class UnknownKeyOrNamespace extends IllegalStateException {
+
+	private static final long serialVersionUID = 1L;
+
+	public UnknownKeyOrNamespace() {
+		super("KvState does not hold any state for key/namespace.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
new file mode 100644
index 0000000..d5ff828
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateID.java
@@ -0,0 +1,35 @@
+/*
+ * 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.queryablestate;
+
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Thrown if no KvState with the given ID cannot found by the server handler.
+ */
+public class UnknownKvStateID extends IllegalStateException {
+
+	private static final long serialVersionUID = 1L;
+
+	public UnknownKvStateID(KvStateID kvStateId) {
+		super("No KvState registered with ID " + Preconditions.checkNotNull(kvStateId, "KvStateID") +
+				" at TaskManager.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.java
new file mode 100644
index 0000000..fd25fae
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/UnknownKvStateKeyGroupLocation.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.queryablestate;
+
+import org.apache.flink.runtime.query.KvStateLocation;
+
+/**
+ * Exception thrown if there is no location information available for the given
+ * key group in a {@link KvStateLocation} instance.
+ */
+public class UnknownKvStateKeyGroupLocation extends Exception {
+
+	private static final long serialVersionUID = 1L;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
new file mode 100644
index 0000000..f42e008
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/AkkaKvStateLocationLookupService.java
@@ -0,0 +1,325 @@
+/*
+ * 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.queryablestate.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.queryablestate.UnknownJobManager;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateMessage;
+import org.apache.flink.util.Preconditions;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
+import akka.dispatch.Mapper;
+import akka.dispatch.Recover;
+import akka.pattern.Patterns;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+import scala.reflect.ClassTag$;
+
+/**
+ * Akka-based {@link KvStateLocationLookupService} that retrieves the current
+ * JobManager address and uses it for lookups.
+ */
+public class AkkaKvStateLocationLookupService implements KvStateLocationLookupService, LeaderRetrievalListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateLocationLookupService.class);
+
+	/** Future returned when no JobManager is available. */
+	private static final Future<ActorGateway> UNKNOWN_JOB_MANAGER = Futures.failed(new UnknownJobManager());
+
+	/** Leader retrieval service to retrieve the current job manager. */
+	private final LeaderRetrievalService leaderRetrievalService;
+
+	/** The actor system used to resolve the JobManager address. */
+	private final ActorSystem actorSystem;
+
+	/** Timeout for JobManager ask-requests. */
+	private final FiniteDuration askTimeout;
+
+	/** Retry strategy factory on future failures. */
+	private final LookupRetryStrategyFactory retryStrategyFactory;
+
+	/** Current job manager future. */
+	private volatile Future<ActorGateway> jobManagerFuture = UNKNOWN_JOB_MANAGER;
+
+	/**
+	 * Creates the Akka-based {@link KvStateLocationLookupService}.
+	 *
+	 * @param leaderRetrievalService Leader retrieval service to use.
+	 * @param actorSystem            Actor system to use.
+	 * @param askTimeout             Timeout for JobManager ask-requests.
+	 * @param retryStrategyFactory   Retry strategy if no JobManager available.
+	 */
+	public AkkaKvStateLocationLookupService(
+			LeaderRetrievalService leaderRetrievalService,
+			ActorSystem actorSystem,
+			FiniteDuration askTimeout,
+			LookupRetryStrategyFactory retryStrategyFactory) {
+
+		this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService, "Leader retrieval service");
+		this.actorSystem = Preconditions.checkNotNull(actorSystem, "Actor system");
+		this.askTimeout = Preconditions.checkNotNull(askTimeout, "Ask Timeout");
+		this.retryStrategyFactory = Preconditions.checkNotNull(retryStrategyFactory, "Retry strategy factory");
+	}
+
+	public void start() {
+		try {
+			leaderRetrievalService.start(this);
+		} catch (Exception e) {
+			LOG.error("Failed to start leader retrieval service", e);
+			throw new RuntimeException(e);
+		}
+	}
+
+	public void shutDown() {
+		try {
+			leaderRetrievalService.stop();
+		} catch (Exception e) {
+			LOG.error("Failed to stop leader retrieval service", e);
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public Future<KvStateLocation> getKvStateLookupInfo(final JobID jobId, final String registrationName) {
+		return getKvStateLookupInfo(jobId, registrationName, retryStrategyFactory.createRetryStrategy());
+	}
+
+	/**
+	 * Returns a future holding the {@link KvStateLocation} for the given job
+	 * and KvState registration name.
+	 *
+	 * <p>If there is currently no JobManager registered with the service, the
+	 * request is retried. The retry behaviour is specified by the
+	 * {@link LookupRetryStrategy} of the lookup service.
+	 *
+	 * @param jobId               JobID the KvState instance belongs to
+	 * @param registrationName    Name under which the KvState has been registered
+	 * @param lookupRetryStrategy Retry strategy to use for retries on UnknownJobManager failures.
+	 * @return Future holding the {@link KvStateLocation}
+	 */
+	@SuppressWarnings("unchecked")
+	private Future<KvStateLocation> getKvStateLookupInfo(
+			final JobID jobId,
+			final String registrationName,
+			final LookupRetryStrategy lookupRetryStrategy) {
+
+		return jobManagerFuture
+				.flatMap(new Mapper<ActorGateway, Future<Object>>() {
+					@Override
+					public Future<Object> apply(ActorGateway jobManager) {
+						// Lookup the KvStateLocation
+						Object msg = new KvStateMessage.LookupKvStateLocation(jobId, registrationName);
+						return jobManager.ask(msg, askTimeout);
+					}
+				}, actorSystem.dispatcher())
+				.mapTo(ClassTag$.MODULE$.<KvStateLocation>apply(KvStateLocation.class))
+				.recoverWith(new Recover<Future<KvStateLocation>>() {
+					@Override
+					public Future<KvStateLocation> recover(Throwable failure) throws Throwable {
+						// If the Future fails with UnknownJobManager, retry
+						// the request. Otherwise all Futures will be failed
+						// during the start up phase, when the JobManager did
+						// not notify this service yet or leadership is lost
+						// intermittently.
+						if (failure instanceof UnknownJobManager && lookupRetryStrategy.tryRetry()) {
+							return Patterns.after(
+									lookupRetryStrategy.getRetryDelay(),
+									actorSystem.scheduler(),
+									actorSystem.dispatcher(),
+									new Callable<Future<KvStateLocation>>() {
+										@Override
+										public Future<KvStateLocation> call() throws Exception {
+											return getKvStateLookupInfo(
+													jobId,
+													registrationName,
+													lookupRetryStrategy);
+										}
+									});
+						} else {
+							return Futures.failed(failure);
+						}
+					}
+				}, actorSystem.dispatcher());
+	}
+
+	@Override
+	public void notifyLeaderAddress(String leaderAddress, final UUID leaderSessionID) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Received leader address notification {}:{}", leaderAddress, leaderSessionID);
+		}
+
+		if (leaderAddress == null) {
+			jobManagerFuture = UNKNOWN_JOB_MANAGER;
+		} else {
+			jobManagerFuture = AkkaUtils.getActorRefFuture(leaderAddress, actorSystem, askTimeout)
+					.map(new Mapper<ActorRef, ActorGateway>() {
+						@Override
+						public ActorGateway apply(ActorRef actorRef) {
+							return new AkkaActorGateway(actorRef, leaderSessionID);
+						}
+					}, actorSystem.dispatcher());
+		}
+	}
+
+	@Override
+	public void handleError(Exception exception) {
+		jobManagerFuture = Futures.failed(exception);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Retry strategy for failed lookups.
+	 *
+	 * <p>Usage:
+	 * <pre>
+	 * LookupRetryStrategy retryStrategy = LookupRetryStrategyFactory.create();
+	 *
+	 * if (retryStrategy.tryRetry()) {
+	 *     // OK to retry
+	 *     FiniteDuration retryDelay = retryStrategy.getRetryDelay();
+	 * }
+	 * </pre>
+	 */
+	public interface LookupRetryStrategy {
+
+		/**
+		 * Returns the current retry.
+		 *
+		 * @return Current retry delay.
+		 */
+		FiniteDuration getRetryDelay();
+
+		/**
+		 * Tries another retry and returns whether it is allowed or not.
+		 *
+		 * @return Whether it is allowed to do another restart or not.
+		 */
+		boolean tryRetry();
+
+	}
+
+	/**
+	 * Factory for retry strategies.
+	 */
+	public interface LookupRetryStrategyFactory {
+
+		/**
+		 * Creates a new retry strategy.
+		 *
+		 * @return The retry strategy.
+		 */
+		LookupRetryStrategy createRetryStrategy();
+
+	}
+
+	/**
+	 * Factory for disabled retries.
+	 */
+	public static class DisabledLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
+
+		private static final DisabledLookupRetryStrategy RETRY_STRATEGY = new DisabledLookupRetryStrategy();
+
+		@Override
+		public LookupRetryStrategy createRetryStrategy() {
+			return RETRY_STRATEGY;
+		}
+
+		private static class DisabledLookupRetryStrategy implements LookupRetryStrategy {
+
+			@Override
+			public FiniteDuration getRetryDelay() {
+				return FiniteDuration.Zero();
+			}
+
+			@Override
+			public boolean tryRetry() {
+				return false;
+			}
+		}
+
+	}
+
+	/**
+	 * Factory for fixed delay retries.
+	 */
+	public static class FixedDelayLookupRetryStrategyFactory implements LookupRetryStrategyFactory {
+
+		private final int maxRetries;
+		private final FiniteDuration retryDelay;
+
+		FixedDelayLookupRetryStrategyFactory(int maxRetries, FiniteDuration retryDelay) {
+			this.maxRetries = maxRetries;
+			this.retryDelay = retryDelay;
+		}
+
+		@Override
+		public LookupRetryStrategy createRetryStrategy() {
+			return new FixedDelayLookupRetryStrategy(maxRetries, retryDelay);
+		}
+
+		private static class FixedDelayLookupRetryStrategy implements LookupRetryStrategy {
+
+			private final Object retryLock = new Object();
+			private final int maxRetries;
+			private final FiniteDuration retryDelay;
+			private int numRetries;
+
+			public FixedDelayLookupRetryStrategy(int maxRetries, FiniteDuration retryDelay) {
+				Preconditions.checkArgument(maxRetries >= 0, "Negative number maximum retries");
+				this.maxRetries = maxRetries;
+				this.retryDelay = Preconditions.checkNotNull(retryDelay, "Retry delay");
+			}
+
+			@Override
+			public FiniteDuration getRetryDelay() {
+				synchronized (retryLock) {
+					return retryDelay;
+				}
+			}
+
+			@Override
+			public boolean tryRetry() {
+				synchronized (retryLock) {
+					if (numRetries < maxRetries) {
+						numRetries++;
+						return true;
+					} else {
+						return false;
+					}
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
new file mode 100644
index 0000000..d456cd7
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClient.java
@@ -0,0 +1,583 @@
+/*
+ * 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.queryablestate.client;
+
+import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.queryablestate.UnknownKvStateID;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.runtime.io.network.netty.NettyBufferPool;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.KvStateRequestStats;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelOption;
+import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler;
+
+import akka.dispatch.Futures;
+
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayDeque;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+
+/**
+ * Netty-based client querying {@link KvStateServer} instances.
+ *
+ * <p>This client can be used by multiple threads concurrently. Operations are
+ * executed asynchronously and return Futures to their result.
+ *
+ * <p>The incoming pipeline looks as follows:
+ * <pre>
+ * Socket.read() -> LengthFieldBasedFrameDecoder -> KvStateServerHandler
+ * </pre>
+ *
+ * <p>Received binary messages are expected to contain a frame length field. Netty's
+ * {@link LengthFieldBasedFrameDecoder} is used to fully receive the frame before
+ * giving it to our {@link KvStateClientHandler}.
+ *
+ * <p>Connections are established and closed by the client. The server only
+ * closes the connection on a fatal failure that cannot be recovered.
+ */
+public class KvStateClient {
+
+	/** Netty's Bootstrap. */
+	private final Bootstrap bootstrap;
+
+	/** Statistics tracker. */
+	private final KvStateRequestStats stats;
+
+	/** Established connections. */
+	private final ConcurrentHashMap<KvStateServerAddress, EstablishedConnection> establishedConnections =
+			new ConcurrentHashMap<>();
+
+	/** Pending connections. */
+	private final ConcurrentHashMap<KvStateServerAddress, PendingConnection> pendingConnections =
+			new ConcurrentHashMap<>();
+
+	/** Atomic shut down flag. */
+	private final AtomicBoolean shutDown = new AtomicBoolean();
+
+	/**
+	 * Creates a client with the specified number of event loop threads.
+	 *
+	 * @param numEventLoopThreads Number of event loop threads (minimum 1).
+	 */
+	public KvStateClient(int numEventLoopThreads, KvStateRequestStats stats) {
+		Preconditions.checkArgument(numEventLoopThreads >= 1, "Non-positive number of event loop threads.");
+		NettyBufferPool bufferPool = new NettyBufferPool(numEventLoopThreads);
+
+		ThreadFactory threadFactory = new ThreadFactoryBuilder()
+				.setDaemon(true)
+				.setNameFormat("Flink KvStateClient Event Loop Thread %d")
+				.build();
+
+		NioEventLoopGroup nioGroup = new NioEventLoopGroup(numEventLoopThreads, threadFactory);
+
+		this.bootstrap = new Bootstrap()
+				.group(nioGroup)
+				.channel(NioSocketChannel.class)
+				.option(ChannelOption.ALLOCATOR, bufferPool)
+				.handler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					protected void initChannel(SocketChannel ch) throws Exception {
+						ch.pipeline()
+								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+								// ChunkedWriteHandler respects Channel writability
+								.addLast(new ChunkedWriteHandler());
+					}
+				});
+
+		this.stats = Preconditions.checkNotNull(stats, "Statistics tracker");
+	}
+
+	/**
+	 * Returns a future holding the serialized request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param serverAddress Address of the server to query
+	 * @param kvStateId ID of the KvState instance to query
+	 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance with
+	 * @return Future holding the serialized result
+	 */
+	public Future<byte[]> getKvState(
+			KvStateServerAddress serverAddress,
+			KvStateID kvStateId,
+			byte[] serializedKeyAndNamespace) {
+
+		if (shutDown.get()) {
+			return Futures.failed(new IllegalStateException("Shut down"));
+		}
+
+		EstablishedConnection connection = establishedConnections.get(serverAddress);
+
+		if (connection != null) {
+			return connection.getKvState(kvStateId, serializedKeyAndNamespace);
+		} else {
+			PendingConnection pendingConnection = pendingConnections.get(serverAddress);
+			if (pendingConnection != null) {
+				// There was a race, use the existing pending connection.
+				return pendingConnection.getKvState(kvStateId, serializedKeyAndNamespace);
+			} else {
+				// We try to connect to the server.
+				PendingConnection pending = new PendingConnection(serverAddress);
+				PendingConnection previous = pendingConnections.putIfAbsent(serverAddress, pending);
+
+				if (previous == null) {
+					// OK, we are responsible to connect.
+					bootstrap.connect(serverAddress.getHost(), serverAddress.getPort())
+							.addListener(pending);
+
+					return pending.getKvState(kvStateId, serializedKeyAndNamespace);
+				} else {
+					// There was a race, use the existing pending connection.
+					return previous.getKvState(kvStateId, serializedKeyAndNamespace);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Shuts down the client and closes all connections.
+	 *
+	 * <p>After a call to this method, all returned futures will be failed.
+	 */
+	public void shutDown() {
+		if (shutDown.compareAndSet(false, true)) {
+			for (Map.Entry<KvStateServerAddress, EstablishedConnection> conn : establishedConnections.entrySet()) {
+				if (establishedConnections.remove(conn.getKey(), conn.getValue())) {
+					conn.getValue().close();
+				}
+			}
+
+			for (Map.Entry<KvStateServerAddress, PendingConnection> conn : pendingConnections.entrySet()) {
+				if (pendingConnections.remove(conn.getKey()) != null) {
+					conn.getValue().close();
+				}
+			}
+
+			if (bootstrap != null) {
+				EventLoopGroup group = bootstrap.group();
+				if (group != null) {
+					group.shutdownGracefully(0, 10, TimeUnit.SECONDS);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Closes the connection to the given server address if it exists.
+	 *
+	 * <p>If there is a request to the server a new connection will be established.
+	 *
+	 * @param serverAddress Target address of the connection to close
+	 */
+	public void closeConnection(KvStateServerAddress serverAddress) {
+		PendingConnection pending = pendingConnections.get(serverAddress);
+		if (pending != null) {
+			pending.close();
+		}
+
+		EstablishedConnection established = establishedConnections.remove(serverAddress);
+		if (established != null) {
+			established.close();
+		}
+	}
+
+	/**
+	 * A pending connection that is in the process of connecting.
+	 */
+	private class PendingConnection implements ChannelFutureListener {
+
+		/** Lock to guard the connect call, channel hand in, etc. */
+		private final Object connectLock = new Object();
+
+		/** Address of the server we are connecting to. */
+		private final KvStateServerAddress serverAddress;
+
+		/** Queue of requests while connecting. */
+		private final ArrayDeque<PendingRequest> queuedRequests = new ArrayDeque<>();
+
+		/** The established connection after the connect succeeds. */
+		private EstablishedConnection established;
+
+		/** Closed flag. */
+		private boolean closed;
+
+		/** Failure cause if something goes wrong. */
+		private Throwable failureCause;
+
+		/**
+		 * Creates a pending connection to the given server.
+		 *
+		 * @param serverAddress Address of the server to connect to.
+		 */
+		private PendingConnection(KvStateServerAddress serverAddress) {
+			this.serverAddress = serverAddress;
+		}
+
+		@Override
+		public void operationComplete(ChannelFuture future) throws Exception {
+			// Callback from the Bootstrap's connect call.
+			if (future.isSuccess()) {
+				handInChannel(future.channel());
+			} else {
+				close(future.cause());
+			}
+		}
+
+		/**
+		 * Returns a future holding the serialized request result.
+		 *
+		 * <p>If the channel has been established, forward the call to the
+		 * established channel, otherwise queue it for when the channel is
+		 * handed in.
+		 *
+		 * @param kvStateId                 ID of the KvState instance to query
+		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
+		 *                                  with
+		 * @return Future holding the serialized result
+		 */
+		public Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
+			synchronized (connectLock) {
+				if (failureCause != null) {
+					return Futures.failed(failureCause);
+				} else if (closed) {
+					return Futures.failed(new ClosedChannelException());
+				} else {
+					if (established != null) {
+						return established.getKvState(kvStateId, serializedKeyAndNamespace);
+					} else {
+						// Queue this and handle when connected
+						PendingRequest pending = new PendingRequest(kvStateId, serializedKeyAndNamespace);
+						queuedRequests.add(pending);
+						return pending.promise.future();
+					}
+				}
+			}
+		}
+
+		/**
+		 * Hands in a channel after a successful connection.
+		 *
+		 * @param channel Channel to hand in
+		 */
+		private void handInChannel(Channel channel) {
+			synchronized (connectLock) {
+				if (closed || failureCause != null) {
+					// Close the channel and we are done. Any queued requests
+					// are removed on the close/failure call and after that no
+					// new ones can be enqueued.
+					channel.close();
+				} else {
+					established = new EstablishedConnection(serverAddress, channel);
+
+					PendingRequest pending;
+					while ((pending = queuedRequests.poll()) != null) {
+						Future<byte[]> resultFuture = established.getKvState(
+								pending.kvStateId,
+								pending.serializedKeyAndNamespace);
+
+						pending.promise.completeWith(resultFuture);
+					}
+
+					// Publish the channel for the general public
+					establishedConnections.put(serverAddress, established);
+					pendingConnections.remove(serverAddress);
+
+					// Check shut down for possible race with shut down. We
+					// don't want any lingering connections after shut down,
+					// which can happen if we don't check this here.
+					if (shutDown.get()) {
+						if (establishedConnections.remove(serverAddress, established)) {
+							established.close();
+						}
+					}
+				}
+			}
+		}
+
+		/**
+		 * Close the connecting channel with a ClosedChannelException.
+		 */
+		private void close() {
+			close(new ClosedChannelException());
+		}
+
+		/**
+		 * Close the connecting channel with an Exception (can be
+		 * <code>null</code>) or forward to the established channel.
+		 */
+		private void close(Throwable cause) {
+			synchronized (connectLock) {
+				if (!closed) {
+					if (failureCause == null) {
+						failureCause = cause;
+					}
+
+					if (established != null) {
+						established.close();
+					} else {
+						PendingRequest pending;
+						while ((pending = queuedRequests.poll()) != null) {
+							pending.promise.tryFailure(cause);
+						}
+					}
+
+					closed = true;
+				}
+			}
+		}
+
+		/**
+		 * A pending request queued while the channel is connecting.
+		 */
+		private final class PendingRequest {
+
+			private final KvStateID kvStateId;
+			private final byte[] serializedKeyAndNamespace;
+			private final Promise<byte[]> promise;
+
+			private PendingRequest(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
+				this.kvStateId = kvStateId;
+				this.serializedKeyAndNamespace = serializedKeyAndNamespace;
+				this.promise = Futures.promise();
+			}
+		}
+
+		@Override
+		public String toString() {
+			synchronized (connectLock) {
+				return "PendingConnection{" +
+						"serverAddress=" + serverAddress +
+						", queuedRequests=" + queuedRequests.size() +
+						", established=" + (established != null) +
+						", closed=" + closed +
+						'}';
+			}
+		}
+	}
+
+	/**
+	 * An established connection that wraps the actual channel instance and is
+	 * registered at the {@link KvStateClientHandler} for callbacks.
+	 */
+	private class EstablishedConnection implements KvStateClientHandlerCallback {
+
+		/** Address of the server we are connected to. */
+		private final KvStateServerAddress serverAddress;
+
+		/** The actual TCP channel. */
+		private final Channel channel;
+
+		/** Pending requests keyed by request ID. */
+		private final ConcurrentHashMap<Long, PromiseAndTimestamp> pendingRequests = new ConcurrentHashMap<>();
+
+		/** Current request number used to assign unique request IDs. */
+		private final AtomicLong requestCount = new AtomicLong();
+
+		/** Reference to a failure that was reported by the channel. */
+		private final AtomicReference<Throwable> failureCause = new AtomicReference<>();
+
+		/**
+		 * Creates an established connection with the given channel.
+		 *
+		 * @param serverAddress Address of the server connected to
+		 * @param channel The actual TCP channel
+		 */
+		EstablishedConnection(KvStateServerAddress serverAddress, Channel channel) {
+			this.serverAddress = Preconditions.checkNotNull(serverAddress, "KvStateServerAddress");
+			this.channel = Preconditions.checkNotNull(channel, "Channel");
+
+			// Add the client handler with the callback
+			channel.pipeline().addLast("KvStateClientHandler", new KvStateClientHandler(this));
+
+			stats.reportActiveConnection();
+		}
+
+		/**
+		 * Close the channel with a ClosedChannelException.
+		 */
+		void close() {
+			close(new ClosedChannelException());
+		}
+
+		/**
+		 * Close the channel with a cause.
+		 *
+		 * @param cause The cause to close the channel with.
+		 * @return Channel close future
+		 */
+		private boolean close(Throwable cause) {
+			if (failureCause.compareAndSet(null, cause)) {
+				channel.close();
+				stats.reportInactiveConnection();
+
+				for (long requestId : pendingRequests.keySet()) {
+					PromiseAndTimestamp pending = pendingRequests.remove(requestId);
+					if (pending != null && pending.promise.tryFailure(cause)) {
+						stats.reportFailedRequest();
+					}
+				}
+
+				return true;
+			}
+
+			return false;
+		}
+
+		/**
+		 * Returns a future holding the serialized request result.
+		 *
+		 * @param kvStateId                 ID of the KvState instance to query
+		 * @param serializedKeyAndNamespace Serialized key and namespace to query KvState instance
+		 *                                  with
+		 * @return Future holding the serialized result
+		 */
+		Future<byte[]> getKvState(KvStateID kvStateId, byte[] serializedKeyAndNamespace) {
+			PromiseAndTimestamp requestPromiseTs = new PromiseAndTimestamp(
+					Futures.<byte[]>promise(),
+					System.nanoTime());
+
+			try {
+				final long requestId = requestCount.getAndIncrement();
+				pendingRequests.put(requestId, requestPromiseTs);
+
+				stats.reportRequest();
+
+				ByteBuf buf = MessageSerializer.serializeKvStateRequest(
+						channel.alloc(),
+						requestId,
+						kvStateId,
+						serializedKeyAndNamespace);
+
+				channel.writeAndFlush(buf).addListener(new ChannelFutureListener() {
+					@Override
+					public void operationComplete(ChannelFuture future) throws Exception {
+						if (!future.isSuccess()) {
+							// Fail promise if not failed to write
+							PromiseAndTimestamp pending = pendingRequests.remove(requestId);
+							if (pending != null && pending.promise.tryFailure(future.cause())) {
+								stats.reportFailedRequest();
+							}
+						}
+					}
+				});
+
+				// Check failure for possible race. We don't want any lingering
+				// promises after a failure, which can happen if we don't check
+				// this here. Note that close is treated as a failure as well.
+				Throwable failure = failureCause.get();
+				if (failure != null) {
+					// Remove from pending requests to guard against concurrent
+					// removal and to make sure that we only count it once as failed.
+					PromiseAndTimestamp p = pendingRequests.remove(requestId);
+					if (p != null && p.promise.tryFailure(failure)) {
+						stats.reportFailedRequest();
+					}
+				}
+			} catch (Throwable t) {
+				requestPromiseTs.promise.tryFailure(t);
+			}
+
+			return requestPromiseTs.promise.future();
+		}
+
+		@Override
+		public void onRequestResult(long requestId, byte[] serializedValue) {
+			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
+			if (pending != null && pending.promise.trySuccess(serializedValue)) {
+				long durationMillis = (System.nanoTime() - pending.timestamp) / 1_000_000;
+				stats.reportSuccessfulRequest(durationMillis);
+			}
+		}
+
+		@Override
+		public void onRequestFailure(long requestId, Throwable cause) {
+			PromiseAndTimestamp pending = pendingRequests.remove(requestId);
+			if (pending != null && pending.promise.tryFailure(cause)) {
+				stats.reportFailedRequest();
+			}
+		}
+
+		@Override
+		public void onFailure(Throwable cause) {
+			if (close(cause)) {
+				// Remove from established channels, otherwise future
+				// requests will be handled by this failed channel.
+				establishedConnections.remove(serverAddress, this);
+			}
+		}
+
+		@Override
+		public String toString() {
+			return "EstablishedConnection{" +
+					"serverAddress=" + serverAddress +
+					", channel=" + channel +
+					", pendingRequests=" + pendingRequests.size() +
+					", requestCount=" + requestCount +
+					", failureCause=" + failureCause +
+					'}';
+		}
+
+		/**
+		 * Pair of promise and a timestamp.
+		 */
+		private class PromiseAndTimestamp {
+
+			private final Promise<byte[]> promise;
+			private final long timestamp;
+
+			public PromiseAndTimestamp(Promise<byte[]> promise, long timestamp) {
+				this.promise = promise;
+				this.timestamp = timestamp;
+			}
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
new file mode 100644
index 0000000..36a2b31
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandler.java
@@ -0,0 +1,107 @@
+/*
+ * 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.queryablestate.client;
+
+import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
+import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.server.KvStateServerHandler;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.util.ReferenceCountUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.channels.ClosedChannelException;
+
+/**
+ * This handler expects responses from {@link KvStateServerHandler}.
+ *
+ * <p>It deserializes the response and calls the registered callback, which is
+ * responsible for actually handling the result (see {@link KvStateClient.EstablishedConnection}).
+ */
+public class KvStateClientHandler extends ChannelInboundHandlerAdapter {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientHandler.class);
+
+	private final KvStateClientHandlerCallback callback;
+
+	/**
+	 * Creates a {@link KvStateClientHandler} with the callback.
+	 *
+	 * @param callback Callback for responses.
+	 */
+	public KvStateClientHandler(KvStateClientHandlerCallback callback) {
+		this.callback = callback;
+	}
+
+	@Override
+	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+		try {
+			ByteBuf buf = (ByteBuf) msg;
+			MessageType msgType = MessageSerializer.deserializeHeader(buf);
+
+			if (msgType == MessageType.REQUEST_RESULT) {
+				KvStateRequestResult result = MessageSerializer.deserializeKvStateRequestResult(buf);
+				callback.onRequestResult(result.getRequestId(), result.getSerializedResult());
+			} else if (msgType == MessageType.REQUEST_FAILURE) {
+				KvStateRequestFailure failure = MessageSerializer.deserializeKvStateRequestFailure(buf);
+				callback.onRequestFailure(failure.getRequestId(), failure.getCause());
+			} else if (msgType == MessageType.SERVER_FAILURE) {
+				throw MessageSerializer.deserializeServerFailure(buf);
+			} else {
+				throw new IllegalStateException("Unexpected response type '" + msgType + "'");
+			}
+		} catch (Throwable t1) {
+			try {
+				callback.onFailure(t1);
+			} catch (Throwable t2) {
+				LOG.error("Failed to notify callback about failure", t2);
+			}
+		} finally {
+			ReferenceCountUtil.release(msg);
+		}
+	}
+
+	@Override
+	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+		try {
+			callback.onFailure(cause);
+		} catch (Throwable t) {
+			LOG.error("Failed to notify callback about failure", t);
+		}
+	}
+
+	@Override
+	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+		// Only the client is expected to close the channel. Otherwise it
+		// indicates a failure. Note that this will be invoked in both cases
+		// though. If the callback closed the channel, the callback must be
+		// ignored.
+		try {
+			callback.onFailure(new ClosedChannelException());
+		} catch (Throwable t) {
+			LOG.error("Failed to notify callback about failure", t);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
new file mode 100644
index 0000000..98718fa
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateClientHandlerCallback.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.client;
+
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+
+/**
+ * Callback for {@link KvStateClientHandler}.
+ */
+public interface KvStateClientHandlerCallback {
+
+	/**
+	 * Called on a successful {@link KvStateRequest}.
+	 *
+	 * @param requestId       ID of the request
+	 * @param serializedValue Serialized value for the request
+	 */
+	void onRequestResult(long requestId, byte[] serializedValue);
+
+	/**
+	 * Called on a failed {@link KvStateRequest}.
+	 *
+	 * @param requestId ID of the request
+	 * @param cause     Cause of the request failure
+	 */
+	void onRequestFailure(long requestId, Throwable cause);
+
+	/**
+	 * Called on any failure, which is not related to a specific request.
+	 *
+	 * <p>This can be for example a caught Exception in the channel pipeline
+	 * or an unexpected channel close.
+	 *
+	 * @param cause Cause of the failure
+	 */
+	void onFailure(Throwable cause);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
new file mode 100644
index 0000000..635cbae
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/KvStateLocationLookupService.java
@@ -0,0 +1,51 @@
+/*
+ * 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.queryablestate.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.query.KvStateLocation;
+
+import scala.concurrent.Future;
+
+/**
+ * {@link KvStateLocation} lookup service.
+ */
+public interface KvStateLocationLookupService {
+
+	/**
+	 * Starts the lookup service.
+	 */
+	void start();
+
+	/**
+	 * Shuts down the lookup service.
+	 */
+	void shutDown();
+
+	/**
+	 * Returns a future holding the {@link KvStateLocation} for the given job
+	 * and KvState registration name.
+	 *
+	 * @param jobId            JobID the KvState instance belongs to
+	 * @param registrationName Name under which the KvState has been registered
+	 * @return Future holding the {@link KvStateLocation}
+	 */
+	Future<KvStateLocation> getKvStateLookupInfo(JobID jobId, String registrationName);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
new file mode 100644
index 0000000..27257d7
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
@@ -0,0 +1,590 @@
+/*
+ * 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.queryablestate.client;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.configuration.AkkaOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.QueryableStateOptions;
+import org.apache.flink.queryablestate.UnknownKeyOrNamespace;
+import org.apache.flink.queryablestate.UnknownKvStateID;
+import org.apache.flink.queryablestate.UnknownKvStateKeyGroupLocation;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.concurrent.Executors;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
+import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceTypeInfo;
+import org.apache.flink.util.Preconditions;
+
+import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
+import akka.dispatch.Mapper;
+import akka.dispatch.Recover;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+/**
+ * Client for queryable state.
+ *
+ * <p>You can mark state as queryable via {@link StateDescriptor#setQueryable(String)}.
+ * The state instance created from this descriptor will be published for queries
+ * when it's created on the TaskManagers and the location will be reported to
+ * the JobManager.
+ *
+ * <p>The client resolves the location of the requested KvState via the
+ * JobManager. Resolved locations are cached. When the server address of the
+ * requested KvState instance is determined, the client sends out a request to
+ * the server.
+ */
+public class QueryableStateClient {
+
+	private static final Logger LOG = LoggerFactory.getLogger(QueryableStateClient.class);
+
+	/**
+	 * {@link KvStateLocation} lookup to resolve the address of KvState instances.
+	 */
+	private final KvStateLocationLookupService lookupService;
+
+	/**
+	 * Network client for queries against {@link KvStateServer} instances.
+	 */
+	private final KvStateClient kvStateClient;
+
+	/**
+	 * Execution context.
+	 */
+	private final ExecutionContext executionContext;
+
+	/**
+	 * Cache for {@link KvStateLocation} instances keyed by job and name.
+	 */
+	private final ConcurrentMap<Tuple2<JobID, String>, Future<KvStateLocation>> lookupCache =
+			new ConcurrentHashMap<>();
+
+	/** This is != null, if we started the actor system. */
+	private final ActorSystem actorSystem;
+
+	private ExecutionConfig executionConfig;
+
+	/**
+	 * Creates a client from the given configuration.
+	 *
+	 * <p>This will create multiple Thread pools: one for the started actor
+	 * system and another for the network client.
+	 *
+	 * @param config Configuration to use.
+	 * @throws Exception Failures are forwarded
+	 */
+	public QueryableStateClient(Configuration config) throws Exception {
+		this(config, HighAvailabilityServicesUtils.createHighAvailabilityServices(
+				config, Executors.directExecutor(), HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION));
+	}
+
+	/**
+	 * Creates a client from the given configuration.
+	 *
+	 * <p>This will create multiple Thread pools: one for the started actor
+	 * system and another for the network client.
+	 *
+	 * @param config Configuration to use.
+	 * @param highAvailabilityServices Service factory for high availability services
+	 * @throws Exception Failures are forwarded
+	 *
+	 * @deprecated This constructor is deprecated and stays only for backwards compatibility. Use the
+	 * {@link #QueryableStateClient(Configuration)} instead.
+	 */
+	@Deprecated
+	public QueryableStateClient(
+			Configuration config,
+			HighAvailabilityServices highAvailabilityServices) throws Exception {
+		Preconditions.checkNotNull(config, "Configuration");
+
+		// Create a leader retrieval service
+		LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID);
+
+		// Get the ask timeout
+		String askTimeoutString = config.getString(AkkaOptions.ASK_TIMEOUT);
+
+		Duration timeout = FiniteDuration.apply(askTimeoutString);
+		if (!timeout.isFinite()) {
+			throw new IllegalConfigurationException(AkkaOptions.ASK_TIMEOUT.key()
+					+ " is not a finite timeout ('" + askTimeoutString + "')");
+		}
+
+		FiniteDuration askTimeout = (FiniteDuration) timeout;
+
+		int lookupRetries = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRIES);
+		int lookupRetryDelayMillis = config.getInteger(QueryableStateOptions.CLIENT_LOOKUP_RETRY_DELAY);
+
+		// Retries if no JobManager is around
+		AkkaKvStateLocationLookupService.LookupRetryStrategyFactory retryStrategy =
+				new AkkaKvStateLocationLookupService.FixedDelayLookupRetryStrategyFactory(
+						lookupRetries,
+						FiniteDuration.apply(lookupRetryDelayMillis, "ms"));
+
+		// Create the actor system
+		@SuppressWarnings("unchecked")
+		Option<Tuple2<String, Object>> remoting = new Some(new Tuple2<>("", 0));
+		this.actorSystem = AkkaUtils.createActorSystem(config, remoting);
+
+		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
+				leaderRetrievalService,
+				actorSystem,
+				askTimeout,
+				retryStrategy);
+
+		int numEventLoopThreads = config.getInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS);
+
+		if (numEventLoopThreads == 0) {
+			numEventLoopThreads = Runtime.getRuntime().availableProcessors();
+		}
+
+		// Create the network client
+		KvStateClient networkClient = new KvStateClient(
+				numEventLoopThreads,
+				new DisabledKvStateRequestStats());
+
+		this.lookupService = lookupService;
+		this.kvStateClient = networkClient;
+		this.executionContext = actorSystem.dispatcher();
+		this.executionConfig = new ExecutionConfig();
+
+		this.lookupService.start();
+	}
+
+	/** Gets the {@link ExecutionConfig}. */
+	public ExecutionConfig getExecutionConfig() {
+		return executionConfig;
+	}
+
+	/** Sets the {@link ExecutionConfig}. */
+	public void setExecutionConfig(ExecutionConfig config) {
+		this.executionConfig = config;
+	}
+
+	/**
+	 * Creates a client.
+	 *
+	 * @param lookupService    Location lookup service
+	 * @param kvStateClient    Network client for queries
+	 * @param executionContext Execution context for futures
+	 */
+	public QueryableStateClient(
+			KvStateLocationLookupService lookupService,
+			KvStateClient kvStateClient,
+			ExecutionContext executionContext) {
+
+		this.lookupService = Preconditions.checkNotNull(lookupService, "KvStateLocationLookupService");
+		this.kvStateClient = Preconditions.checkNotNull(kvStateClient, "KvStateClient");
+		this.executionContext = Preconditions.checkNotNull(executionContext, "ExecutionContext");
+		this.actorSystem = null;
+
+		this.lookupService.start();
+	}
+
+	/**
+	 * Returns the execution context of this client.
+	 *
+	 * @return The execution context used by the client.
+	 */
+	public ExecutionContext getExecutionContext() {
+		return executionContext;
+	}
+
+	/**
+	 * Shuts down the client and all components.
+	 */
+	public void shutDown() {
+		try {
+			lookupService.shutDown();
+		} catch (Throwable t) {
+			LOG.error("Failed to shut down KvStateLookupService", t);
+		}
+
+		try {
+			kvStateClient.shutDown();
+		} catch (Throwable t) {
+			LOG.error("Failed to shut down KvStateClient", t);
+		}
+
+		if (actorSystem != null) {
+			try {
+				actorSystem.shutdown();
+			} catch (Throwable t) {
+				LOG.error("Failed to shut down ActorSystem", t);
+			}
+		}
+	}
+
+	/**
+	 * Returns a future holding the serialized request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param jobId                     JobID of the job the queryable state
+	 *                                  belongs to
+	 * @param queryableStateName        Name under which the state is queryable
+	 * @param keyHashCode               Integer hash code of the key (result of
+	 *                                  a call to {@link Object#hashCode()}
+	 * @param serializedKeyAndNamespace Serialized key and namespace to query
+	 *                                  KvState instance with
+	 * @return Future holding the serialized result
+	 */
+	@SuppressWarnings("unchecked")
+	public Future<byte[]> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final int keyHashCode,
+			final byte[] serializedKeyAndNamespace) {
+
+		return getKvState(jobId, queryableStateName, keyHashCode, serializedKeyAndNamespace, false)
+				.recoverWith(new Recover<Future<byte[]>>() {
+					@Override
+					public Future<byte[]> recover(Throwable failure) throws Throwable {
+						if (failure instanceof UnknownKvStateID ||
+								failure instanceof UnknownKvStateKeyGroupLocation ||
+								failure instanceof UnknownKvStateLocation ||
+								failure instanceof ConnectException) {
+							// These failures are likely to be caused by out-of-sync
+							// KvStateLocation. Therefore we retry this query and
+							// force look up the location.
+							return getKvState(
+									jobId,
+									queryableStateName,
+									keyHashCode,
+									serializedKeyAndNamespace,
+									true);
+						} else {
+							return Futures.failed(failure);
+						}
+					}
+				}, executionContext);
+	}
+
+	/**
+	 * Returns a future holding the request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param jobId                     JobID of the job the queryable state belongs to.
+	 * @param queryableStateName        Name under which the state is queryable.
+	 * @param key			            The key we are interested in.
+	 * @param keyTypeHint				A {@link TypeHint} used to extract the type of the key.
+	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
+	 * @return Future holding the result.
+	 */
+	@PublicEvolving
+	public <K, V> Future<V> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final TypeHint<K> keyTypeHint,
+			final StateDescriptor<?, V> stateDescriptor) {
+
+		Preconditions.checkNotNull(keyTypeHint);
+
+		TypeInformation<K> keyTypeInfo = keyTypeHint.getTypeInfo();
+		return getKvState(jobId, queryableStateName, key, keyTypeInfo, stateDescriptor);
+	}
+
+	/**
+	 * Returns a future holding the request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param jobId                     JobID of the job the queryable state belongs to.
+	 * @param queryableStateName        Name under which the state is queryable.
+	 * @param key			            The key we are interested in.
+	 * @param keyTypeInfo				The {@link TypeInformation} of the key.
+	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
+	 * @return Future holding the result.
+	 */
+	@PublicEvolving
+	public <K, V> Future<V> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final TypeInformation<K> keyTypeInfo,
+			final StateDescriptor<?, V> stateDescriptor) {
+
+		Preconditions.checkNotNull(keyTypeInfo);
+
+		return getKvState(jobId, queryableStateName, key, VoidNamespace.INSTANCE,
+				keyTypeInfo, VoidNamespaceTypeInfo.INSTANCE, stateDescriptor);
+	}
+
+	/**
+	 * Returns a future holding the request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param jobId                     JobID of the job the queryable state belongs to.
+	 * @param queryableStateName        Name under which the state is queryable.
+	 * @param key			            The key that the state we request is associated with.
+	 * @param namespace					The namespace of the state.
+	 * @param keyTypeInfo				The {@link TypeInformation} of the keys.
+	 * @param namespaceTypeInfo			The {@link TypeInformation} of the namespace.
+	 * @param stateDescriptor			The {@link StateDescriptor} of the state we want to query.
+	 * @return Future holding the result.
+	 */
+	@PublicEvolving
+	public <K, V, N> Future<V> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final N namespace,
+			final TypeInformation<K> keyTypeInfo,
+			final TypeInformation<N> namespaceTypeInfo,
+			final StateDescriptor<?, V> stateDescriptor) {
+
+		Preconditions.checkNotNull(stateDescriptor);
+
+		// initialize the value serializer based on the execution config.
+		stateDescriptor.initializeSerializerUnlessSet(executionConfig);
+		TypeSerializer<V> stateSerializer = stateDescriptor.getSerializer();
+
+		return getKvState(jobId, queryableStateName, key,
+				namespace, keyTypeInfo, namespaceTypeInfo, stateSerializer);
+	}
+
+	/**
+	 * Returns a future holding the request result.
+	 *
+	 * <p>If the server does not serve a KvState instance with the given ID,
+	 * the Future will be failed with a {@link UnknownKvStateID}.
+	 *
+	 * <p>If the KvState instance does not hold any data for the given key
+	 * and namespace, the Future will be failed with a {@link UnknownKeyOrNamespace}.
+	 *
+	 * <p>All other failures are forwarded to the Future.
+	 *
+	 * @param jobId                     JobID of the job the queryable state belongs to.
+	 * @param queryableStateName        Name under which the state is queryable.
+	 * @param key			            The key that the state we request is associated with.
+	 * @param namespace					The namespace of the state.
+	 * @param keyTypeInfo				The {@link TypeInformation} of the keys.
+	 * @param namespaceTypeInfo			The {@link TypeInformation} of the namespace.
+	 * @param stateSerializer			The {@link TypeSerializer} of the state we want to query.
+	 * @return Future holding the result.
+	 */
+	@PublicEvolving
+	public <K, V, N> Future<V> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final K key,
+			final N namespace,
+			final TypeInformation<K> keyTypeInfo,
+			final TypeInformation<N> namespaceTypeInfo,
+			final TypeSerializer<V> stateSerializer) {
+
+		Preconditions.checkNotNull(queryableStateName);
+
+		Preconditions.checkNotNull(key);
+		Preconditions.checkNotNull(namespace);
+
+		Preconditions.checkNotNull(keyTypeInfo);
+		Preconditions.checkNotNull(namespaceTypeInfo);
+		Preconditions.checkNotNull(stateSerializer);
+
+		if (stateSerializer instanceof ListSerializer) {
+			throw new IllegalArgumentException("ListState is not supported out-of-the-box yet.");
+		}
+
+		TypeSerializer<K> keySerializer = keyTypeInfo.createSerializer(executionConfig);
+		TypeSerializer<N> namespaceSerializer = namespaceTypeInfo.createSerializer(executionConfig);
+
+		final byte[] serializedKeyAndNamespace;
+		try {
+			serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+					key,
+					keySerializer,
+					namespace,
+					namespaceSerializer);
+		} catch (IOException e) {
+			return Futures.failed(e);
+		}
+
+		return getKvState(jobId, queryableStateName, key.hashCode(), serializedKeyAndNamespace)
+				.flatMap(new Mapper<byte[], Future<V>>() {
+					@Override
+					public Future<V> apply(byte[] parameter) {
+						try {
+							return Futures.successful(
+									KvStateSerializer.deserializeValue(parameter, stateSerializer));
+						} catch (IOException e) {
+							return Futures.failed(e);
+						}
+					}
+				}, executionContext);
+	}
+
+	/**
+	 * Returns a future holding the serialized request result.
+	 *
+	 * @param jobId                     JobID of the job the queryable state
+	 *                                  belongs to
+	 * @param queryableStateName        Name under which the state is queryable
+	 * @param keyHashCode               Integer hash code of the key (result of
+	 *                                  a call to {@link Object#hashCode()}
+	 * @param serializedKeyAndNamespace Serialized key and namespace to query
+	 *                                  KvState instance with
+	 * @param forceLookup               Flag to force lookup of the {@link KvStateLocation}
+	 * @return Future holding the serialized result
+	 */
+	private Future<byte[]> getKvState(
+			final JobID jobId,
+			final String queryableStateName,
+			final int keyHashCode,
+			final byte[] serializedKeyAndNamespace,
+			boolean forceLookup) {
+
+		return getKvStateLookupInfo(jobId, queryableStateName, forceLookup)
+				.flatMap(new Mapper<KvStateLocation, Future<byte[]>>() {
+					@Override
+					public Future<byte[]> apply(KvStateLocation lookup) {
+						int keyGroupIndex = KeyGroupRangeAssignment.computeKeyGroupForKeyHash(keyHashCode, lookup.getNumKeyGroups());
+
+						KvStateServerAddress serverAddress = lookup.getKvStateServerAddress(keyGroupIndex);
+						if (serverAddress == null) {
+							return Futures.failed(new UnknownKvStateKeyGroupLocation());
+						} else {
+							// Query server
+							KvStateID kvStateId = lookup.getKvStateID(keyGroupIndex);
+							return kvStateClient.getKvState(serverAddress, kvStateId, serializedKeyAndNamespace);
+						}
+					}
+				}, executionContext);
+	}
+
+	/**
+	 * Lookup the {@link KvStateLocation} for the given job and queryable state
+	 * name.
+	 *
+	 * <p>The job manager will be queried for the location only if forced or no
+	 * cached location can be found. There are no guarantees about
+	 *
+	 * @param jobId              JobID the state instance belongs to.
+	 * @param queryableStateName Name under which the state instance has been published.
+	 * @param forceUpdate        Flag to indicate whether to force a update via the lookup service.
+	 * @return Future holding the KvStateLocation
+	 */
+	private Future<KvStateLocation> getKvStateLookupInfo(
+			JobID jobId,
+			final String queryableStateName,
+			boolean forceUpdate) {
+
+		if (forceUpdate) {
+			Future<KvStateLocation> lookupFuture = lookupService
+					.getKvStateLookupInfo(jobId, queryableStateName);
+			lookupCache.put(new Tuple2<>(jobId, queryableStateName), lookupFuture);
+			return lookupFuture;
+		} else {
+			Tuple2<JobID, String> cacheKey = new Tuple2<>(jobId, queryableStateName);
+			final Future<KvStateLocation> cachedFuture = lookupCache.get(cacheKey);
+
+			if (cachedFuture == null) {
+				Future<KvStateLocation> lookupFuture = lookupService
+						.getKvStateLookupInfo(jobId, queryableStateName);
+
+				Future<KvStateLocation> previous = lookupCache.putIfAbsent(cacheKey, lookupFuture);
+				if (previous == null) {
+					return lookupFuture;
+				} else {
+					return previous;
+				}
+			} else {
+				// do not retain futures which failed as they will remain in
+				// the cache even if the error cause is not present any more
+				// and a new lookup may succeed
+				if (cachedFuture.isCompleted() &&
+						cachedFuture.value().get().isFailure()) {
+					// issue a new lookup
+					Future<KvStateLocation> lookupFuture = lookupService
+							.getKvStateLookupInfo(jobId, queryableStateName);
+
+					// replace the existing one if it has not been replaced yet
+					// otherwise return the one in the cache
+					if (lookupCache.replace(cacheKey, cachedFuture, lookupFuture)) {
+						return lookupFuture;
+					} else {
+						return lookupCache.get(cacheKey);
+					}
+				} else {
+					return cachedFuture;
+				}
+			}
+		}
+	}
+
+}


[06/14] flink git commit: [FLINK-7769][QS] Move queryable state outside the runtime.

Posted by kk...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
new file mode 100644
index 0000000..15a5ff6
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateITCase.java
@@ -0,0 +1,101 @@
+/*
+ * 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.queryablestate.itcases;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.QueryableStateOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+
+import org.apache.curator.test.TestingServer;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.rules.TemporaryFolder;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Base class with the cluster configuration for the tests on the NON-HA mode.
+ */
+public abstract class HAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
+
+	private static final int NUM_JMS = 2;
+	private static final int NUM_TMS = 4;
+	private static final int NUM_SLOTS_PER_TM = 4;
+
+	private static TestingServer zkServer;
+	private static TemporaryFolder temporaryFolder;
+
+	@BeforeClass
+	public static void setup() {
+		try {
+			zkServer = new TestingServer();
+			temporaryFolder = new TemporaryFolder();
+			temporaryFolder.create();
+
+			Configuration config = new Configuration();
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
+			config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true);
+			config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2);
+			config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2);
+			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString());
+			config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString());
+			config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+
+			cluster = new TestingCluster(config, false);
+			cluster.start();
+
+			testActorSystem = AkkaUtils.createDefaultActorSystem();
+
+			// verify that we are in HA mode
+			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.ZOOKEEPER);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() {
+		if (cluster != null) {
+			cluster.stop();
+			cluster.awaitTermination();
+		}
+
+		testActorSystem.shutdown();
+		testActorSystem.awaitTermination();
+
+		try {
+			zkServer.stop();
+			zkServer.close();
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+
+		temporaryFolder.delete();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseFsBackend.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseFsBackend.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseFsBackend.java
new file mode 100644
index 0000000..a2d3ad0
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseFsBackend.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.itcases;
+
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Several integration tests for queryable state using the {@link FsStateBackend}.
+ */
+public class HAQueryableStateITCaseFsBackend extends HAAbstractQueryableStateITCase {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Override
+	protected AbstractStateBackend createStateBackend() throws Exception {
+		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseRocksDBBackend.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseRocksDBBackend.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseRocksDBBackend.java
new file mode 100644
index 0000000..fda1171
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateITCaseRocksDBBackend.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.itcases;
+
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
+ */
+public class HAQueryableStateITCaseRocksDBBackend extends HAAbstractQueryableStateITCase {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Override
+	protected AbstractStateBackend createStateBackend() throws Exception {
+		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
new file mode 100644
index 0000000..907e8a3
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/KVStateRequestSerializerRocksDBTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.queryablestate.itcases;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.contrib.streaming.state.PredefinedOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
+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.apache.flink.runtime.state.internal.InternalMapState;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+
+import java.io.File;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Additional tests for the serialization and deserialization using
+ * the KvStateSerializer with a RocksDB state back-end.
+ */
+public final class KVStateRequestSerializerRocksDBTest {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	/**
+	 * Extension of {@link RocksDBKeyedStateBackend} to make {@link
+	 * #createListState(TypeSerializer, ListStateDescriptor)} public for use in
+	 * the tests.
+	 *
+	 * @param <K> key type
+	 */
+	static final class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {
+
+		RocksDBKeyedStateBackend2(
+				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,
+				final ExecutionConfig executionConfig) throws Exception {
+
+			super(operatorIdentifier, userCodeClassLoader,
+				instanceBasePath,
+				dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer,
+				numberOfKeyGroups, keyGroupRange, executionConfig, false);
+		}
+
+		@Override
+		public <N, T> InternalListState<N, T> createListState(
+			final TypeSerializer<N> namespaceSerializer,
+			final ListStateDescriptor<T> stateDesc) throws Exception {
+
+			return super.createListState(namespaceSerializer, stateDesc);
+		}
+	}
+
+	/**
+	 * Tests list serialization and deserialization match.
+	 *
+	 * @see KvStateRequestSerializerTest#testListSerialization()
+	 * KvStateRequestSerializerTest#testListSerialization() using the heap state back-end
+	 * test
+	 */
+	@Test
+	public void testListSerialization() throws Exception {
+		final long key = 0L;
+
+		// objects for RocksDB state list serialisation
+		DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
+		dbOptions.setCreateIfMissing(true);
+		ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
+		final RocksDBKeyedStateBackend2<Long> longHeapKeyedStateBackend =
+			new RocksDBKeyedStateBackend2<>(
+				"no-op",
+				ClassLoader.getSystemClassLoader(),
+				temporaryFolder.getRoot(),
+				dbOptions,
+				columnFamilyOptions,
+				mock(TaskKvStateRegistry.class),
+				LongSerializer.INSTANCE,
+				1, new KeyGroupRange(0, 0),
+				new ExecutionConfig()
+			);
+		longHeapKeyedStateBackend.restore(null);
+		longHeapKeyedStateBackend.setCurrentKey(key);
+
+		final InternalListState<VoidNamespace, Long> listState = longHeapKeyedStateBackend
+			.createListState(VoidNamespaceSerializer.INSTANCE,
+				new ListStateDescriptor<>("test", LongSerializer.INSTANCE));
+
+		KvStateRequestSerializerTest.testListSerialization(key, listState);
+	}
+
+	/**
+	 * Tests map serialization and deserialization match.
+	 *
+	 * @see KvStateRequestSerializerTest#testMapSerialization()
+	 * KvStateRequestSerializerTest#testMapSerialization() using the heap state back-end
+	 * test
+	 */
+	@Test
+	public void testMapSerialization() throws Exception {
+		final long key = 0L;
+
+		// objects for RocksDB state list serialisation
+		DBOptions dbOptions = PredefinedOptions.DEFAULT.createDBOptions();
+		dbOptions.setCreateIfMissing(true);
+		ColumnFamilyOptions columnFamilyOptions = PredefinedOptions.DEFAULT.createColumnOptions();
+		final RocksDBKeyedStateBackend<Long> longHeapKeyedStateBackend =
+			new RocksDBKeyedStateBackend<>(
+				"no-op",
+				ClassLoader.getSystemClassLoader(),
+				temporaryFolder.getRoot(),
+				dbOptions,
+				columnFamilyOptions,
+				mock(TaskKvStateRegistry.class),
+				LongSerializer.INSTANCE,
+				1, new KeyGroupRange(0, 0),
+				new ExecutionConfig(),
+				false);
+		longHeapKeyedStateBackend.restore(null);
+		longHeapKeyedStateBackend.setCurrentKey(key);
+
+		final InternalMapState<VoidNamespace, Long, String> mapState = (InternalMapState<VoidNamespace, Long, String>)
+				longHeapKeyedStateBackend.getPartitionedState(
+						VoidNamespace.INSTANCE,
+						VoidNamespaceSerializer.INSTANCE,
+						new MapStateDescriptor<>("test", LongSerializer.INSTANCE, StringSerializer.INSTANCE));
+
+		KvStateRequestSerializerTest.testMapSerialization(key, mapState);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
new file mode 100644
index 0000000..c52acc8
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateITCase.java
@@ -0,0 +1,81 @@
+/*
+ * 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.queryablestate.itcases;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.QueryableStateOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Base class with the cluster configuration for the tests on the HA mode.
+ */
+public abstract class NonHAAbstractQueryableStateITCase extends AbstractQueryableStateITCase {
+
+	private static final int NUM_TMS = 2;
+	private static final int NUM_SLOTS_PER_TM = 4;
+
+	@BeforeClass
+	public static void setup() {
+		try {
+			Configuration config = new Configuration();
+			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM);
+			config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1);
+			config.setBoolean(QueryableStateOptions.SERVER_ENABLE, true);
+			config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1);
+
+			cluster = new TestingCluster(config, false);
+			cluster.start(true);
+
+			testActorSystem = AkkaUtils.createDefaultActorSystem();
+
+			// verify that we are not in HA mode
+			Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.NONE);
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDown() {
+		try {
+			cluster.shutdown();
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+
+		if (testActorSystem != null) {
+			testActorSystem.shutdown();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseFsBackend.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseFsBackend.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseFsBackend.java
new file mode 100644
index 0000000..caa315a
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseFsBackend.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.itcases;
+
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Several integration tests for queryable state using the {@link FsStateBackend}.
+ */
+public class NonHAQueryableStateITCaseFsBackend extends NonHAAbstractQueryableStateITCase {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Override
+	protected AbstractStateBackend createStateBackend() throws Exception {
+		return new FsStateBackend(temporaryFolder.newFolder().toURI().toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseRocksDBBackend.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseRocksDBBackend.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseRocksDBBackend.java
new file mode 100644
index 0000000..10e9b57
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateITCaseRocksDBBackend.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.queryablestate.itcases;
+
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Several integration tests for queryable state using the {@link RocksDBStateBackend}.
+ */
+public class NonHAQueryableStateITCaseRocksDBBackend extends NonHAAbstractQueryableStateITCase {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	@Override
+	protected AbstractStateBackend createStateBackend() throws Exception {
+		return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
new file mode 100644
index 0000000..d9a41a1
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/AkkaKvStateLocationLookupServiceTest.java
@@ -0,0 +1,399 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.queryablestate.UnknownJobManager;
+import org.apache.flink.queryablestate.client.AkkaKvStateLocationLookupService;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.akka.FlinkUntypedActor;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateMessage.LookupKvStateLocation;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.Props;
+import akka.actor.Status;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link AkkaKvStateLocationLookupService}.
+ */
+public class AkkaKvStateLocationLookupServiceTest extends TestLogger {
+
+	/** The default timeout. */
+	private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
+
+	/** Test actor system shared between the tests. */
+	private static ActorSystem testActorSystem;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		testActorSystem = AkkaUtils.createLocalActorSystem(new Configuration());
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (testActorSystem != null) {
+			testActorSystem.shutdown();
+		}
+	}
+
+	/**
+	 * Tests responses if no leader notification has been reported or leadership
+	 * has been lost (leaderAddress = <code>null</code>).
+	 */
+	@Test
+	public void testNoJobManagerRegistered() throws Exception {
+		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
+			null,
+			null);
+		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
+
+		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
+				leaderRetrievalService,
+				testActorSystem,
+				TIMEOUT,
+				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
+
+		lookupService.start();
+
+		//
+		// No leader registered initially => fail with UnknownJobManager
+		//
+		try {
+			JobID jobId = new JobID();
+			String name = "coffee";
+
+			Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(jobId, name);
+
+			Await.result(locationFuture, TIMEOUT);
+			fail("Did not throw expected Exception");
+		} catch (UnknownJobManager ignored) {
+			// Expected
+		}
+
+		assertEquals("Received unexpected lookup", 0, received.size());
+
+		//
+		// Leader registration => communicate with new leader
+		//
+		UUID leaderSessionId = HighAvailabilityServices.DEFAULT_LEADER_ID;
+		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "tea");
+
+		ActorRef testActor = LookupResponseActor.create(received, leaderSessionId, expected);
+
+		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
+
+		// Notify the service about a leader
+		leaderRetrievalService.notifyListener(testActorAddress, leaderSessionId);
+
+		JobID jobId = new JobID();
+		String name = "tea";
+
+		// Verify that the leader response is handled
+		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, name), TIMEOUT);
+		assertEquals(expected, location);
+
+		// Verify that the correct message was sent to the leader
+		assertEquals(1, received.size());
+
+		verifyLookupMsg(received.poll(), jobId, name);
+
+		//
+		// Leader loss => fail with UnknownJobManager
+		//
+		leaderRetrievalService.notifyListener(null, null);
+
+		try {
+			Future<KvStateLocation> locationFuture = lookupService
+					.getKvStateLookupInfo(new JobID(), "coffee");
+
+			Await.result(locationFuture, TIMEOUT);
+			fail("Did not throw expected Exception");
+		} catch (UnknownJobManager ignored) {
+			// Expected
+		}
+
+		// No new messages received
+		assertEquals(0, received.size());
+	}
+
+	/**
+	 * Tests that messages are properly decorated with the leader session ID.
+	 */
+	@Test
+	public void testLeaderSessionIdChange() throws Exception {
+		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
+			"localhost",
+			HighAvailabilityServices.DEFAULT_LEADER_ID);
+		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
+
+		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
+				leaderRetrievalService,
+				testActorSystem,
+				TIMEOUT,
+				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
+
+		lookupService.start();
+
+		// Create test actors with random leader session IDs
+		KvStateLocation expected1 = new KvStateLocation(new JobID(), new JobVertexID(), 8282, "salt");
+		UUID leaderSessionId1 = UUID.randomUUID();
+		ActorRef testActor1 = LookupResponseActor.create(received, leaderSessionId1, expected1);
+		String testActorAddress1 = AkkaUtils.getAkkaURL(testActorSystem, testActor1);
+
+		KvStateLocation expected2 = new KvStateLocation(new JobID(), new JobVertexID(), 22321, "pepper");
+		UUID leaderSessionId2 = UUID.randomUUID();
+		ActorRef testActor2 = LookupResponseActor.create(received, leaderSessionId1, expected2);
+		String testActorAddress2 = AkkaUtils.getAkkaURL(testActorSystem, testActor2);
+
+		JobID jobId = new JobID();
+
+		//
+		// Notify about first leader
+		//
+		leaderRetrievalService.notifyListener(testActorAddress1, leaderSessionId1);
+
+		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(jobId, "rock"), TIMEOUT);
+		assertEquals(expected1, location);
+
+		assertEquals(1, received.size());
+		verifyLookupMsg(received.poll(), jobId, "rock");
+
+		//
+		// Notify about second leader
+		//
+		leaderRetrievalService.notifyListener(testActorAddress2, leaderSessionId2);
+
+		location = Await.result(lookupService.getKvStateLookupInfo(jobId, "roll"), TIMEOUT);
+		assertEquals(expected2, location);
+
+		assertEquals(1, received.size());
+		verifyLookupMsg(received.poll(), jobId, "roll");
+	}
+
+	/**
+	 * Tests that lookups are retried when no leader notification is available.
+	 */
+	@Test
+	public void testRetryOnUnknownJobManager() throws Exception {
+		final Queue<AkkaKvStateLocationLookupService.LookupRetryStrategy> retryStrategies = new LinkedBlockingQueue<>();
+
+		AkkaKvStateLocationLookupService.LookupRetryStrategyFactory retryStrategy =
+				new AkkaKvStateLocationLookupService.LookupRetryStrategyFactory() {
+					@Override
+					public AkkaKvStateLocationLookupService.LookupRetryStrategy createRetryStrategy() {
+						return retryStrategies.poll();
+					}
+				};
+
+		final TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
+			null,
+			null);
+
+		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
+				leaderRetrievalService,
+				testActorSystem,
+				TIMEOUT,
+				retryStrategy);
+
+		lookupService.start();
+
+		//
+		// Test call to retry
+		//
+		final AtomicBoolean hasRetried = new AtomicBoolean();
+		retryStrategies.add(
+				new AkkaKvStateLocationLookupService.LookupRetryStrategy() {
+					@Override
+					public FiniteDuration getRetryDelay() {
+						return FiniteDuration.Zero();
+					}
+
+					@Override
+					public boolean tryRetry() {
+						if (hasRetried.compareAndSet(false, true)) {
+							return true;
+						}
+						return false;
+					}
+				});
+
+		Future<KvStateLocation> locationFuture = lookupService.getKvStateLookupInfo(new JobID(), "yessir");
+
+		Await.ready(locationFuture, TIMEOUT);
+		assertTrue("Did not retry ", hasRetried.get());
+
+		//
+		// Test leader notification after retry
+		//
+		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
+
+		KvStateLocation expected = new KvStateLocation(new JobID(), new JobVertexID(), 12122, "garlic");
+		ActorRef testActor = LookupResponseActor.create(received, null, expected);
+		final String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
+
+		retryStrategies.add(new AkkaKvStateLocationLookupService.LookupRetryStrategy() {
+			@Override
+			public FiniteDuration getRetryDelay() {
+				return FiniteDuration.apply(100, TimeUnit.MILLISECONDS);
+			}
+
+			@Override
+			public boolean tryRetry() {
+				leaderRetrievalService.notifyListener(testActorAddress, HighAvailabilityServices.DEFAULT_LEADER_ID);
+				return true;
+			}
+		});
+
+		KvStateLocation location = Await.result(lookupService.getKvStateLookupInfo(new JobID(), "yessir"), TIMEOUT);
+		assertEquals(expected, location);
+	}
+
+	@Test
+	public void testUnexpectedResponseType() throws Exception {
+		TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService(
+			"localhost",
+			HighAvailabilityServices.DEFAULT_LEADER_ID);
+		Queue<LookupKvStateLocation> received = new LinkedBlockingQueue<>();
+
+		AkkaKvStateLocationLookupService lookupService = new AkkaKvStateLocationLookupService(
+				leaderRetrievalService,
+				testActorSystem,
+				TIMEOUT,
+				new AkkaKvStateLocationLookupService.DisabledLookupRetryStrategyFactory());
+
+		lookupService.start();
+
+		// Create test actors with random leader session IDs
+		String expected = "unexpected-response-type";
+		ActorRef testActor = LookupResponseActor.create(received, null, expected);
+		String testActorAddress = AkkaUtils.getAkkaURL(testActorSystem, testActor);
+
+		leaderRetrievalService.notifyListener(testActorAddress, null);
+
+		try {
+			Await.result(lookupService.getKvStateLookupInfo(new JobID(), "spicy"), TIMEOUT);
+			fail("Did not throw expected Exception");
+		} catch (Throwable ignored) {
+			// Expected
+		}
+	}
+
+	private static final class LookupResponseActor extends FlinkUntypedActor {
+
+		/** Received lookup messages. */
+		private final Queue<LookupKvStateLocation> receivedLookups;
+
+		/** Responses on KvStateMessage.LookupKvStateLocation messages. */
+		private final Queue<Object> lookupResponses;
+
+		/** The leader session ID. */
+		private UUID leaderSessionId;
+
+		public LookupResponseActor(
+				Queue<LookupKvStateLocation> receivedLookups,
+				UUID leaderSessionId, Object... lookupResponses) {
+
+			this.receivedLookups = Preconditions.checkNotNull(receivedLookups, "Received lookups");
+			this.leaderSessionId = leaderSessionId;
+			this.lookupResponses = new ArrayDeque<>();
+
+			if (lookupResponses != null) {
+				for (Object resp : lookupResponses) {
+					this.lookupResponses.add(resp);
+				}
+			}
+		}
+
+		@Override
+		public void handleMessage(Object message) throws Exception {
+			if (message instanceof LookupKvStateLocation) {
+				// Add to received lookups queue
+				receivedLookups.add((LookupKvStateLocation) message);
+
+				Object msg = lookupResponses.poll();
+				if (msg != null) {
+					if (msg instanceof Throwable) {
+						sender().tell(new Status.Failure((Throwable) msg), self());
+					} else {
+						sender().tell(new Status.Success(msg), self());
+					}
+				}
+			} else if (message instanceof UUID) {
+				this.leaderSessionId = (UUID) message;
+			} else {
+				LOG.debug("Received unhandled message: {}", message);
+			}
+		}
+
+		@Override
+		protected UUID getLeaderSessionID() {
+			return leaderSessionId;
+		}
+
+		private static ActorRef create(
+				Queue<LookupKvStateLocation> receivedLookups,
+				UUID leaderSessionId,
+				Object... lookupResponses) {
+
+			return testActorSystem.actorOf(Props.create(
+					LookupResponseActor.class,
+					receivedLookups,
+					leaderSessionId,
+					lookupResponses));
+		}
+	}
+
+	private static void verifyLookupMsg(
+			LookupKvStateLocation lookUpMsg,
+			JobID expectedJobId,
+			String expectedName) {
+
+		assertNotNull(lookUpMsg);
+		assertEquals(expectedJobId, lookUpMsg.getJobId());
+		assertEquals(expectedName, lookUpMsg.getRegistrationName());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
new file mode 100644
index 0000000..0b97bda
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientHandlerTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.queryablestate.client.KvStateClientHandler;
+import org.apache.flink.queryablestate.client.KvStateClientHandlerCallback;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+
+import org.junit.Test;
+
+import java.nio.channels.ClosedChannelException;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+/**
+ * Tests for {@link KvStateClientHandler}.
+ */
+public class KvStateClientHandlerTest {
+
+	/**
+	 * Tests that on reads the expected callback methods are called and read
+	 * buffers are recycled.
+	 */
+	@Test
+	public void testReadCallbacksAndBufferRecycling() throws Exception {
+		KvStateClientHandlerCallback callback = mock(KvStateClientHandlerCallback.class);
+
+		EmbeddedChannel channel = new EmbeddedChannel(new KvStateClientHandler(callback));
+
+		//
+		// Request success
+		//
+		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
+				channel.alloc(),
+				1222112277,
+				new byte[0]);
+		buf.skipBytes(4); // skip frame length
+
+		// Verify callback
+		channel.writeInbound(buf);
+		verify(callback, times(1)).onRequestResult(eq(1222112277L), any(byte[].class));
+		assertEquals("Buffer not recycled", 0, buf.refCnt());
+
+		//
+		// Request failure
+		//
+		buf = MessageSerializer.serializeKvStateRequestFailure(
+				channel.alloc(),
+				1222112278,
+				new RuntimeException("Expected test Exception"));
+		buf.skipBytes(4); // skip frame length
+
+		// Verify callback
+		channel.writeInbound(buf);
+		verify(callback, times(1)).onRequestFailure(eq(1222112278L), any(RuntimeException.class));
+		assertEquals("Buffer not recycled", 0, buf.refCnt());
+
+		//
+		// Server failure
+		//
+		buf = MessageSerializer.serializeServerFailure(
+				channel.alloc(),
+				new RuntimeException("Expected test Exception"));
+		buf.skipBytes(4); // skip frame length
+
+		// Verify callback
+		channel.writeInbound(buf);
+		verify(callback, times(1)).onFailure(any(RuntimeException.class));
+
+		//
+		// Unexpected messages
+		//
+		buf = channel.alloc().buffer(4).writeInt(1223823);
+
+		// Verify callback
+		channel.writeInbound(buf);
+		verify(callback, times(2)).onFailure(any(IllegalStateException.class));
+		assertEquals("Buffer not recycled", 0, buf.refCnt());
+
+		//
+		// Exception caught
+		//
+		channel.pipeline().fireExceptionCaught(new RuntimeException("Expected test Exception"));
+		verify(callback, times(3)).onFailure(any(RuntimeException.class));
+
+		//
+		// Channel inactive
+		//
+		channel.pipeline().fireChannelInactive();
+		verify(callback, times(4)).onFailure(any(ClosedChannelException.class));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
new file mode 100644
index 0000000..a2850b3
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateClientTest.java
@@ -0,0 +1,752 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.queryablestate.client.KvStateClient;
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.queryablestate.server.KvStateServerImpl;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateRegistry;
+import org.apache.flink.runtime.query.KvStateServer;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.netty.AtomicKvStateRequestStats;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+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.InternalKvState;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.util.NetUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer;
+import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link KvStateClient}.
+ */
+public class KvStateClientTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KvStateClientTest.class);
+
+	// Thread pool for client bootstrap (shared between tests)
+	private static final NioEventLoopGroup NIO_GROUP = new NioEventLoopGroup();
+
+	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(100, TimeUnit.SECONDS);
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (NIO_GROUP != null) {
+			NIO_GROUP.shutdownGracefully();
+		}
+	}
+
+	/**
+	 * Tests simple queries, of which half succeed and half fail.
+	 */
+	@Test
+	public void testSimpleRequests() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateClient client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new KvStateClient(1, stats);
+
+			// Random result
+			final byte[] expected = new byte[1024];
+			ThreadLocalRandom.current().nextBytes(expected);
+
+			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.add((ByteBuf) msg);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			List<Future<byte[]>> futures = new ArrayList<>();
+
+			int numQueries = 1024;
+
+			for (int i = 0; i < numQueries; i++) {
+				futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
+			}
+
+			// Respond to messages
+			Exception testException = new RuntimeException("Expected test Exception");
+
+			for (int i = 0; i < numQueries; i++) {
+				ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				assertNotNull("Receive timed out", buf);
+
+				Channel ch = channel.get();
+				assertNotNull("Channel not active", ch);
+
+				assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+				KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
+
+				buf.release();
+
+				if (i % 2 == 0) {
+					ByteBuf response = MessageSerializer.serializeKvStateRequestResult(
+							serverChannel.alloc(),
+							request.getRequestId(),
+							expected);
+
+					ch.writeAndFlush(response);
+				} else {
+					ByteBuf response = MessageSerializer.serializeKvStateRequestFailure(
+							serverChannel.alloc(),
+							request.getRequestId(),
+							testException);
+
+					ch.writeAndFlush(response);
+				}
+			}
+
+			for (int i = 0; i < numQueries; i++) {
+				if (i % 2 == 0) {
+					byte[] serializedResult = Await.result(futures.get(i), deadline.timeLeft());
+					assertArrayEquals(expected, serializedResult);
+				} else {
+					try {
+						Await.result(futures.get(i), deadline.timeLeft());
+						fail("Did not throw expected Exception");
+					} catch (RuntimeException ignored) {
+						// Expected
+					}
+				}
+			}
+
+			assertEquals(numQueries, stats.getNumRequests());
+			int expectedRequests = numQueries / 2;
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != expectedRequests ||
+					stats.getNumFailed() != expectedRequests)) {
+				Thread.sleep(100);
+			}
+
+			assertEquals(expectedRequests, stats.getNumSuccessful());
+			assertEquals(expectedRequests, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a request to an unavailable host is failed with ConnectException.
+	 */
+	@Test
+	public void testRequestUnavailableHost() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+		KvStateClient client = null;
+
+		try {
+			client = new KvStateClient(1, stats);
+
+			int availablePort = NetUtils.getAvailablePort();
+
+			KvStateServerAddress serverAddress = new KvStateServerAddress(
+					InetAddress.getLocalHost(),
+					availablePort);
+
+			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
+
+			try {
+				Await.result(future, deadline.timeLeft());
+				fail("Did not throw expected ConnectException");
+			} catch (ConnectException ignored) {
+				// Expected
+			}
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			assertEquals("Channel leak", 0, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Multiple threads concurrently fire queries.
+	 */
+	@Test
+	public void testConcurrentQueries() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		ExecutorService executor = null;
+		KvStateClient client = null;
+		Channel serverChannel = null;
+
+		final byte[] serializedResult = new byte[1024];
+		ThreadLocalRandom.current().nextBytes(serializedResult);
+
+		try {
+			int numQueryTasks = 4;
+			final int numQueriesPerTask = 1024;
+
+			executor = Executors.newFixedThreadPool(numQueryTasks);
+
+			client = new KvStateClient(1, stats);
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					ByteBuf buf = (ByteBuf) msg;
+					assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+					KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
+
+					buf.release();
+
+					ByteBuf response = MessageSerializer.serializeKvStateRequestResult(
+							ctx.alloc(),
+							request.getRequestId(),
+							serializedResult);
+
+					ctx.channel().writeAndFlush(response);
+				}
+			});
+
+			final KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			final KvStateClient finalClient = client;
+			Callable<List<Future<byte[]>>> queryTask = new Callable<List<Future<byte[]>>>() {
+				@Override
+				public List<Future<byte[]>> call() throws Exception {
+					List<Future<byte[]>> results = new ArrayList<>(numQueriesPerTask);
+
+					for (int i = 0; i < numQueriesPerTask; i++) {
+						results.add(finalClient.getKvState(
+								serverAddress,
+								new KvStateID(),
+								new byte[0]));
+					}
+
+					return results;
+				}
+			};
+
+			// Submit query tasks
+			List<java.util.concurrent.Future<List<Future<byte[]>>>> futures = new ArrayList<>();
+			for (int i = 0; i < numQueryTasks; i++) {
+				futures.add(executor.submit(queryTask));
+			}
+
+			// Verify results
+			for (java.util.concurrent.Future<List<Future<byte[]>>> future : futures) {
+				List<Future<byte[]>> results = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+				for (Future<byte[]> result : results) {
+					byte[] actual = Await.result(result, deadline.timeLeft());
+					assertArrayEquals(serializedResult, actual);
+				}
+			}
+
+			int totalQueries = numQueryTasks * numQueriesPerTask;
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && stats.getNumSuccessful() != totalQueries) {
+				Thread.sleep(100);
+			}
+
+			assertEquals(totalQueries, stats.getNumRequests());
+			assertEquals(totalQueries, stats.getNumSuccessful());
+		} finally {
+			if (executor != null) {
+				executor.shutdown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			if (client != null) {
+				client.shutDown();
+			}
+
+			assertEquals("Channel leak", 0, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a server failure closes the connection and removes it from
+	 * the established connections.
+	 */
+	@Test
+	public void testFailureClosesChannel() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateClient client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new KvStateClient(1, stats);
+
+			final LinkedBlockingQueue<ByteBuf> received = new LinkedBlockingQueue<>();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.add((ByteBuf) msg);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			// Requests
+			List<Future<byte[]>> futures = new ArrayList<>();
+			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
+			futures.add(client.getKvState(serverAddress, new KvStateID(), new byte[0]));
+
+			ByteBuf buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			assertNotNull("Receive timed out", buf);
+			buf.release();
+
+			buf = received.poll(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+			assertNotNull("Receive timed out", buf);
+			buf.release();
+
+			assertEquals(1, stats.getNumConnections());
+
+			Channel ch = channel.get();
+			assertNotNull("Channel not active", ch);
+
+			// Respond with failure
+			ch.writeAndFlush(MessageSerializer.serializeServerFailure(
+					serverChannel.alloc(),
+					new RuntimeException("Expected test server failure")));
+
+			try {
+				Await.result(futures.remove(0), deadline.timeLeft());
+				fail("Did not throw expected server failure");
+			} catch (RuntimeException ignored) {
+				// Expected
+			}
+
+			try {
+				Await.result(futures.remove(0), deadline.timeLeft());
+				fail("Did not throw expected server failure");
+			} catch (RuntimeException ignored) {
+				// Expected
+			}
+
+			assertEquals(0, stats.getNumConnections());
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
+					stats.getNumFailed() != 2)) {
+				Thread.sleep(100);
+			}
+
+			assertEquals(2, stats.getNumRequests());
+			assertEquals(0, stats.getNumSuccessful());
+			assertEquals(2, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests that a server channel close, closes the connection and removes it
+	 * from the established connections.
+	 */
+	@Test
+	public void testServerClosesChannel() throws Exception {
+		Deadline deadline = TEST_TIMEOUT.fromNow();
+		AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats();
+
+		KvStateClient client = null;
+		Channel serverChannel = null;
+
+		try {
+			client = new KvStateClient(1, stats);
+
+			final AtomicBoolean received = new AtomicBoolean();
+			final AtomicReference<Channel> channel = new AtomicReference<>();
+
+			serverChannel = createServerChannel(new ChannelInboundHandlerAdapter() {
+				@Override
+				public void channelActive(ChannelHandlerContext ctx) throws Exception {
+					channel.set(ctx.channel());
+				}
+
+				@Override
+				public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+					received.set(true);
+				}
+			});
+
+			KvStateServerAddress serverAddress = getKvStateServerAddress(serverChannel);
+
+			// Requests
+			Future<byte[]> future = client.getKvState(serverAddress, new KvStateID(), new byte[0]);
+
+			while (!received.get() && deadline.hasTimeLeft()) {
+				Thread.sleep(50);
+			}
+			assertTrue("Receive timed out", received.get());
+
+			assertEquals(1, stats.getNumConnections());
+
+			channel.get().close().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS);
+
+			try {
+				Await.result(future, deadline.timeLeft());
+				fail("Did not throw expected server failure");
+			} catch (ClosedChannelException ignored) {
+				// Expected
+			}
+
+			assertEquals(0, stats.getNumConnections());
+
+			// Counts can take some time to propagate
+			while (deadline.hasTimeLeft() && (stats.getNumSuccessful() != 0 ||
+					stats.getNumFailed() != 1)) {
+				Thread.sleep(100);
+			}
+
+			assertEquals(1, stats.getNumRequests());
+			assertEquals(0, stats.getNumSuccessful());
+			assertEquals(1, stats.getNumFailed());
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			if (serverChannel != null) {
+				serverChannel.close();
+			}
+
+			assertEquals("Channel leak", 0, stats.getNumConnections());
+		}
+	}
+
+	/**
+	 * Tests multiple clients querying multiple servers until 100k queries have
+	 * been processed. At this point, the client is shut down and its verified
+	 * that all ongoing requests are failed.
+	 */
+	@Test
+	public void testClientServerIntegration() throws Exception {
+		// Config
+		final int numServers = 2;
+		final int numServerEventLoopThreads = 2;
+		final int numServerQueryThreads = 2;
+
+		final int numClientEventLoopThreads = 4;
+		final int numClientsTasks = 8;
+
+		final int batchSize = 16;
+
+		final int numKeyGroups = 1;
+
+		AbstractStateBackend abstractBackend = new MemoryStateBackend();
+		KvStateRegistry dummyRegistry = new KvStateRegistry();
+		DummyEnvironment dummyEnv = new DummyEnvironment("test", 1, 0);
+		dummyEnv.setKvStateRegistry(dummyRegistry);
+
+		AbstractKeyedStateBackend<Integer> backend = abstractBackend.createKeyedStateBackend(
+				dummyEnv,
+				new JobID(),
+				"test_op",
+				IntSerializer.INSTANCE,
+				numKeyGroups,
+				new KeyGroupRange(0, 0),
+				dummyRegistry.createTaskRegistry(new JobID(), new JobVertexID()));
+
+		final FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+
+		AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats();
+
+		KvStateClient client = null;
+		ExecutorService clientTaskExecutor = null;
+		final KvStateServer[] server = new KvStateServer[numServers];
+
+		try {
+			client = new KvStateClient(numClientEventLoopThreads, clientStats);
+			clientTaskExecutor = Executors.newFixedThreadPool(numClientsTasks);
+
+			// Create state
+			ValueStateDescriptor<Integer> desc = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE);
+			desc.setQueryable("any");
+
+			// Create servers
+			KvStateRegistry[] registry = new KvStateRegistry[numServers];
+			AtomicKvStateRequestStats[] serverStats = new AtomicKvStateRequestStats[numServers];
+			final KvStateID[] ids = new KvStateID[numServers];
+
+			for (int i = 0; i < numServers; i++) {
+				registry[i] = new KvStateRegistry();
+				serverStats[i] = new AtomicKvStateRequestStats();
+				server[i] = new KvStateServerImpl(
+						InetAddress.getLocalHost(),
+						0,
+						numServerEventLoopThreads,
+						numServerQueryThreads,
+						registry[i],
+						serverStats[i]);
+
+				server[i].start();
+
+				backend.setCurrentKey(1010 + i);
+
+				// Value per server
+				ValueState<Integer> state = backend.getPartitionedState(VoidNamespace.INSTANCE,
+						VoidNamespaceSerializer.INSTANCE,
+						desc);
+
+				state.update(201 + i);
+
+				// we know it must be a KvStat but this is not exposed to the user via 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);
+			}
+
+			final KvStateClient finalClient = client;
+			Callable<Void> queryTask = new Callable<Void>() {
+				@Override
+				public Void call() throws Exception {
+					while (true) {
+						if (Thread.interrupted()) {
+							throw new InterruptedException();
+						}
+
+						// Random server permutation
+						List<Integer> random = new ArrayList<>();
+						for (int j = 0; j < batchSize; j++) {
+							random.add(j);
+						}
+						Collections.shuffle(random);
+
+						// Dispatch queries
+						List<Future<byte[]>> futures = new ArrayList<>(batchSize);
+
+						for (int j = 0; j < batchSize; j++) {
+							int targetServer = random.get(j) % numServers;
+
+							byte[] serializedKeyAndNamespace = KvStateSerializer.serializeKeyAndNamespace(
+									1010 + targetServer,
+									IntSerializer.INSTANCE,
+									VoidNamespace.INSTANCE,
+									VoidNamespaceSerializer.INSTANCE);
+
+							futures.add(finalClient.getKvState(
+									server[targetServer].getAddress(),
+									ids[targetServer],
+									serializedKeyAndNamespace));
+						}
+
+						// Verify results
+						for (int j = 0; j < batchSize; j++) {
+							int targetServer = random.get(j) % numServers;
+
+							Future<byte[]> future = futures.get(j);
+							byte[] buf = Await.result(future, timeout);
+							int value = KvStateSerializer.deserializeValue(buf, IntSerializer.INSTANCE);
+							assertEquals(201 + targetServer, value);
+						}
+					}
+				}
+			};
+
+			// Submit tasks
+			List<java.util.concurrent.Future<Void>> taskFutures = new ArrayList<>();
+			for (int i = 0; i < numClientsTasks; i++) {
+				taskFutures.add(clientTaskExecutor.submit(queryTask));
+			}
+
+			long numRequests;
+			while ((numRequests = clientStats.getNumRequests()) < 100_000) {
+				Thread.sleep(100);
+				LOG.info("Number of requests {}/100_000", numRequests);
+			}
+
+			// Shut down
+			client.shutDown();
+
+			for (java.util.concurrent.Future<Void> future : taskFutures) {
+				try {
+					future.get();
+					fail("Did not throw expected Exception after shut down");
+				} catch (ExecutionException t) {
+					if (t.getCause() instanceof ClosedChannelException ||
+							t.getCause() instanceof IllegalStateException) {
+						// Expected
+					} else {
+						t.printStackTrace();
+						fail("Failed with unexpected Exception type: " + t.getClass().getName());
+					}
+				}
+			}
+
+			assertEquals("Connection leak (client)", 0, clientStats.getNumConnections());
+			for (int i = 0; i < numServers; i++) {
+				boolean success = false;
+				int numRetries = 0;
+				while (!success) {
+					try {
+						assertEquals("Connection leak (server)", 0, serverStats[i].getNumConnections());
+						success = true;
+					} catch (Throwable t) {
+						if (numRetries < 10) {
+							LOG.info("Retrying connection leak check (server)");
+							Thread.sleep((numRetries + 1) * 50);
+							numRetries++;
+						} else {
+							throw t;
+						}
+					}
+				}
+			}
+		} finally {
+			if (client != null) {
+				client.shutDown();
+			}
+
+			for (int i = 0; i < numServers; i++) {
+				if (server[i] != null) {
+					server[i].shutDown();
+				}
+			}
+
+			if (clientTaskExecutor != null) {
+				clientTaskExecutor.shutdown();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	private Channel createServerChannel(final ChannelHandler... handlers) throws UnknownHostException, InterruptedException {
+		ServerBootstrap bootstrap = new ServerBootstrap()
+				// Bind address and port
+				.localAddress(InetAddress.getLocalHost(), 0)
+				// NIO server channels
+				.group(NIO_GROUP)
+				.channel(NioServerSocketChannel.class)
+				// See initializer for pipeline details
+				.childHandler(new ChannelInitializer<SocketChannel>() {
+					@Override
+					protected void initChannel(SocketChannel ch) throws Exception {
+						ch.pipeline()
+								.addLast(new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4))
+								.addLast(handlers);
+					}
+				});
+
+		return bootstrap.bind().sync().channel();
+	}
+
+	private KvStateServerAddress getKvStateServerAddress(Channel serverChannel) {
+		InetSocketAddress localAddress = (InetSocketAddress) serverChannel.localAddress();
+
+		return new KvStateServerAddress(localAddress.getAddress(), localAddress.getPort());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29a6e995/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
new file mode 100644
index 0000000..f28ca68
--- /dev/null
+++ b/flink-queryable-state/flink-queryable-state-java/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.queryablestate.network;
+
+import org.apache.flink.queryablestate.messages.KvStateRequest;
+import org.apache.flink.queryablestate.messages.KvStateRequestFailure;
+import org.apache.flink.queryablestate.messages.KvStateRequestResult;
+import org.apache.flink.queryablestate.network.messages.MessageSerializer;
+import org.apache.flink.queryablestate.network.messages.MessageType;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.netty.message.KvStateSerializer;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.buffer.UnpooledByteBufAllocator;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link KvStateSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class KvStateRequestSerializerTest {
+
+	private final ByteBufAllocator alloc = UnpooledByteBufAllocator.DEFAULT;
+
+	@Parameterized.Parameters
+	public static Collection<Boolean> parameters() {
+		return Arrays.asList(false, true);
+	}
+
+	@Parameterized.Parameter
+	public boolean async;
+
+	/**
+	 * Tests KvState request serialization.
+	 */
+	@Test
+	public void testKvStateRequestSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 1337L;
+		KvStateID kvStateId = new KvStateID();
+		byte[] serializedKeyAndNamespace = randomByteArray(1024);
+
+		ByteBuf buf = MessageSerializer.serializeKvStateRequest(
+				alloc,
+				requestId,
+				kvStateId,
+				serializedKeyAndNamespace);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+		KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(requestId, request.getRequestId());
+		assertEquals(kvStateId, request.getKvStateId());
+		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
+	}
+
+	/**
+	 * Tests KvState request serialization with zero-length serialized key and namespace.
+	 */
+	@Test
+	public void testKvStateRequestSerializationWithZeroLengthKeyAndNamespace() throws Exception {
+		byte[] serializedKeyAndNamespace = new byte[0];
+
+		ByteBuf buf = MessageSerializer.serializeKvStateRequest(
+				alloc,
+				1823,
+				new KvStateID(),
+				serializedKeyAndNamespace);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST, MessageSerializer.deserializeHeader(buf));
+		KvStateRequest request = MessageSerializer.deserializeKvStateRequest(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertArrayEquals(serializedKeyAndNamespace, request.getSerializedKeyAndNamespace());
+	}
+
+	/**
+	 * Tests that we don't try to be smart about <code>null</code> key and namespace.
+	 * They should be treated explicitly.
+	 */
+	@Test(expected = NullPointerException.class)
+	public void testNullPointerExceptionOnNullSerializedKeyAndNamepsace() throws Exception {
+		new KvStateRequest(0, new KvStateID(), null);
+	}
+
+	/**
+	 * Tests KvState request result serialization.
+	 */
+	@Test
+	public void testKvStateRequestResultSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 72727278L;
+		byte[] serializedResult = randomByteArray(1024);
+
+		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
+				alloc,
+				requestId,
+				serializedResult);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestResult request = MessageSerializer.deserializeKvStateRequestResult(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(requestId, request.getRequestId());
+
+		assertArrayEquals(serializedResult, request.getSerializedResult());
+	}
+
+	/**
+	 * Tests KvState request result serialization with zero-length serialized result.
+	 */
+	@Test
+	public void testKvStateRequestResultSerializationWithZeroLengthSerializedResult() throws Exception {
+		byte[] serializedResult = new byte[0];
+
+		ByteBuf buf = MessageSerializer.serializeKvStateRequestResult(
+				alloc,
+				72727278,
+				serializedResult);
+
+		int frameLength = buf.readInt();
+
+		assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestResult request = MessageSerializer.deserializeKvStateRequestResult(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertArrayEquals(serializedResult, request.getSerializedResult());
+	}
+
+	/**
+	 * Tests that we don't try to be smart about <code>null</code> results.
+	 * They should be treated explicitly.
+	 */
+	@Test(expected = NullPointerException.class)
+	public void testNullPointerExceptionOnNullSerializedResult() throws Exception {
+		new KvStateRequestResult(0, null);
+	}
+
+	/**
+	 * Tests KvState request failure serialization.
+	 */
+	@Test
+	public void testKvStateRequestFailureSerialization() throws Exception {
+		long requestId = Integer.MAX_VALUE + 1111222L;
+		IllegalStateException cause = new IllegalStateException("Expected test");
+
+		ByteBuf buf = MessageSerializer.serializeKvStateRequestFailure(
+				alloc,
+				requestId,
+				cause);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf));
+		KvStateRequestFailure request = MessageSerializer.deserializeKvStateRequestFailure(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(requestId, request.getRequestId());
+		assertEquals(cause.getClass(), request.getCause().getClass());
+		assertEquals(cause.getMessage(), request.getCause().getMessage());
+	}
+
+	/**
+	 * Tests KvState server failure serialization.
+	 */
+	@Test
+	public void testServerFailureSerialization() throws Exception {
+		IllegalStateException cause = new IllegalStateException("Expected test");
+
+		ByteBuf buf = MessageSerializer.serializeServerFailure(alloc, cause);
+
+		int frameLength = buf.readInt();
+		assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf));
+		Throwable request = MessageSerializer.deserializeServerFailure(buf);
+		assertEquals(buf.readerIndex(), frameLength + 4);
+
+		assertEquals(cause.getClass(), request.getClass());
+		assertEquals(cause.getMessage(), request.getMessage());
+	}
+
+	private byte[] randomByteArray(int capacity) {
+		byte[] bytes = new byte[capacity];
+		ThreadLocalRandom.current().nextBytes(bytes);
+		return bytes;
+	}
+}